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 2020/11/09 08:22:59 UTC

[GitHub] [spark] leanken opened a new pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

leanken opened a new pull request #30296:
URL: https://github.com/apache/spark/pull/30296


   ### What changes were proposed in this pull request?
   
   element_at with CreateArray not respect one based index.
   
   repo step:
   
   ```
   var df = spark.sql("select element_at(array(3, 2, 1), 0)")
   df.printSchema()
   
   df = spark.sql("select element_at(array(3, 2, 1), 1)")
   df.printSchema()
   
   df = spark.sql("select element_at(array(3, 2, 1), 2)")
   df.printSchema()
   
   df = spark.sql("select element_at(array(3, 2, 1), 3)")
   df.printSchema()
   
   root
   – element_at(array(3, 2, 1), 0): integer (nullable = false)
   
   root
   – element_at(array(3, 2, 1), 1): integer (nullable = false)
   
   root
   – element_at(array(3, 2, 1), 2): integer (nullable = false)
   
   root
   – element_at(array(3, 2, 1), 3): integer (nullable = true)
   
   correct answer should be 
   0 true which is outOfBounds return default true.
   1 false
   2 false
   3 false
   
   ```
   
   For expression eval, it respect the oneBasedIndex, but within checking the nullable, it calculates with zeroBasedIndex using `computeNullabilityFromArray`.
   
   
   ### Why are the changes needed?
   
   Correctness issue.
   
   
   ### Does this PR introduce any user-facing change?
   
   No.
   
   
   ### How was this patch tested?
   
   Added UT and existing UT.


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

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-723915550


   Merged build finished. Test FAILed.


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

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


[GitHub] [spark] cloud-fan commented on a change in pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30296:
URL: https://github.com/apache/spark/pull/30296#discussion_r520260092



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala
##########
@@ -1401,6 +1401,40 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession {
     assert(e3.message.contains(errorMsg3))
   }
 
+  test("SPARK-33391: element_at with CreateArray") {
+    // element_at should use one-based index and support negative index.
+    // valid index for array(1, 2, 3) should be 1,2,3,-1,-2,-3
+    var df = OneRowRelation().selectExpr("element_at(array(1, 2, 3), 1)")
+    assert(!df.schema.head.nullable)
+    checkAnswer(
+      df,
+      Seq(Row(1))
+    )
+
+    df = OneRowRelation().selectExpr("element_at(array(1, 2, 3), -1)")
+    assert(!df.schema.head.nullable)
+    checkAnswer(
+      df,
+      Seq(Row(3))
+    )
+
+    df = OneRowRelation().selectExpr("element_at(array(1, 2, 3), 3)")
+    assert(!df.schema.head.nullable)
+    checkAnswer(
+      df,
+      Seq(Row(3))
+    )
+
+    // 0 is not a valid index, return nullable = false since it throws exception.

Review comment:
       can we test `4, -4` as well?




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

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


[GitHub] [spark] gatorsmile commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
gatorsmile commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-725532778


   @leanken If possible, please also mention when we introduced the bug.
   
   This is a regression introduced in https://issues.apache.org/jira/browse/SPARK-26965. Thus, Spark 2.4 is safe.
   
   


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

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


[GitHub] [spark] cloud-fan commented on a change in pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30296:
URL: https://github.com/apache/spark/pull/30296#discussion_r519785383



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
##########
@@ -1966,7 +1966,20 @@ case class ElementAt(left: Expression, right: Expression)
   }
 
   override def nullable: Boolean = left.dataType match {
-    case _: ArrayType => computeNullabilityFromArray(left, right)
+    case _: ArrayType =>
+      def specialNormalizeIndex: (Int, Int) => Int = {
+        (arrayLength: Int, index: Int) => {
+          if (index < 0) {
+            arrayLength + index
+          } else if (index == 0) {

Review comment:
       why not
   ```
   if (index <= 0) {
     arrayLength + index
   } ...
   ```




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

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


[GitHub] [spark] leanken commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
leanken commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-723849548


   @cloud-fan FYI.


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

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


[GitHub] [spark] leanken commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
leanken commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-728800539


   > Hi @leanken !
   > I'd ask maybe Is the PR description for `spark.sql("select element_at(array(3, 2, 1), 0)").printSchema()` correct?
   > I ran the reproducers in the PR description and it shows a different result in the master branch.
   > 
   > ```scala
   > scala> spark.sql("select element_at(array(3, 2, 1), 0)").printSchema()
   > root
   >  |-- element_at(array(3, 2, 1), 0): integer (nullable = false)
   > ```
   > 
   > It returned `nullable = false`, but in the PR description says that `true` is expected.
   > Or could you please correct me if I missed something??
   > Thanks :)
   
   Oh. the PR desc is outdated, thanks for the mentions. And as for the correct answer, it should return false.
   because spark.sql("select element_at(array(3, 2, 1), 0)").collect will return a runtime exception when ansi mode is on or off, in this case, it should be false, because if nullable = true, it should return null.
   


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

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


[GitHub] [spark] itholic commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
itholic commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-728769565


   Hi @leanken ! 
   I'd ask maybe Is the PR description for `spark.sql("select element_at(array(3, 2, 1), 0)").printSchema()` correct?
   I ran the reproducers in the PR description and it shows a different result in the master branch.
   
   ```scala
   scala> spark.sql("select element_at(array(3, 2, 1), 0)").printSchema()
   root
    |-- element_at(array(3, 2, 1), 0): integer (nullable = false)
   ```
   
   It returned `nullable = false`, but in the PR description says that `true` is expected.
   Or could you please correct me if I missed something??
   Thanks :)


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

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


[GitHub] [spark] SparkQA commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724415796


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35426/
   


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

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


[GitHub] [spark] SparkQA commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724492803


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35445/
   


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

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724506494


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/130831/
   Test FAILed.


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

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


[GitHub] [spark] cloud-fan commented on a change in pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30296:
URL: https://github.com/apache/spark/pull/30296#discussion_r520277078



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala
##########
@@ -1122,11 +1122,16 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper
     val a = AttributeReference("a", IntegerType, nullable = false)()
     val b = AttributeReference("b", IntegerType, nullable = true)()
     val array = CreateArray(a :: b :: Nil)
-    assert(!ElementAt(array, Literal(0)).nullable)
-    assert(ElementAt(array, Literal(1)).nullable)
-    assert(!ElementAt(array, Subtract(Literal(2), Literal(2))).nullable)
+    assert(!ElementAt(array, Literal(1)).nullable)
+    assert(ElementAt(array, Literal(2)).nullable)
+    assert(!ElementAt(array, Subtract(Literal(2), Literal(1))).nullable)

Review comment:
       let's test valid negative ordinals.




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

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


[GitHub] [spark] leanken commented on a change in pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
leanken commented on a change in pull request #30296:
URL: https://github.com/apache/spark/pull/30296#discussion_r519669993



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
##########
@@ -1966,7 +1966,7 @@ case class ElementAt(left: Expression, right: Expression)
   }
 
   override def nullable: Boolean = left.dataType match {
-    case _: ArrayType => computeNullabilityFromArray(left, right)
+    case _: ArrayType => computeNullabilityFromArray(left, right, isOneBasedIndex = true)

Review comment:
       ```
   _FUNC_(array, index) - Returns element of array at given (1-based) index. If index < 0,
         accesses elements from the last to the first. Returns NULL if the index exceeds the length
         of the array.
   ```




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

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


[GitHub] [spark] AmplabJenkins commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-723915550






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

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


[GitHub] [spark] cloud-fan commented on a change in pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30296:
URL: https://github.com/apache/spark/pull/30296#discussion_r519785078



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
##########
@@ -1966,7 +1966,20 @@ case class ElementAt(left: Expression, right: Expression)
   }
 
   override def nullable: Boolean = left.dataType match {
-    case _: ArrayType => computeNullabilityFromArray(left, right)
+    case _: ArrayType =>
+      def specialNormalizeIndex: (Int, Int) => Int = {
+        (arrayLength: Int, index: Int) => {
+          if (index < 0) {
+            arrayLength + index

Review comment:
       this can still be negative and fail, right?




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

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-723891344






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

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


[GitHub] [spark] SparkQA removed a comment on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724438673


   **[Test build #130831 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130831/testReport)** for PR 30296 at commit [`7d09d8c`](https://github.com/apache/spark/commit/7d09d8c74a6e0b72ff7fc2ea881a3c437a4236b7).


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

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


[GitHub] [spark] leanken commented on a change in pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
leanken commented on a change in pull request #30296:
URL: https://github.com/apache/spark/pull/30296#discussion_r520260502



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala
##########
@@ -1401,6 +1401,40 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession {
     assert(e3.message.contains(errorMsg3))
   }
 
+  test("SPARK-33391: element_at with CreateArray") {
+    // element_at should use one-based index and support negative index.
+    // valid index for array(1, 2, 3) should be 1,2,3,-1,-2,-3
+    var df = OneRowRelation().selectExpr("element_at(array(1, 2, 3), 1)")
+    assert(!df.schema.head.nullable)
+    checkAnswer(
+      df,
+      Seq(Row(1))
+    )
+
+    df = OneRowRelation().selectExpr("element_at(array(1, 2, 3), -1)")
+    assert(!df.schema.head.nullable)
+    checkAnswer(
+      df,
+      Seq(Row(3))
+    )
+
+    df = OneRowRelation().selectExpr("element_at(array(1, 2, 3), 3)")
+    assert(!df.schema.head.nullable)
+    checkAnswer(
+      df,
+      Seq(Row(3))
+    )
+
+    // 0 is not a valid index, return nullable = false since it throws exception.

Review comment:
       OK




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

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


[GitHub] [spark] SparkQA commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724421775


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35426/
   


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

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


[GitHub] [spark] SparkQA commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724438673


   **[Test build #130831 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130831/testReport)** for PR 30296 at commit [`7d09d8c`](https://github.com/apache/spark/commit/7d09d8c74a6e0b72ff7fc2ea881a3c437a4236b7).


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

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


[GitHub] [spark] AmplabJenkins commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724014853






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

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


[GitHub] [spark] SparkQA commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724140919


   **[Test build #130784 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130784/testReport)** for PR 30296 at commit [`c0bf2f2`](https://github.com/apache/spark/commit/c0bf2f20ea7fec771a7444d617afd6ca11306aa3).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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

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


[GitHub] [spark] SparkQA commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724014835


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35393/
   


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

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


[GitHub] [spark] AmplabJenkins commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724421797






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

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


[GitHub] [spark] cloud-fan commented on a change in pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30296:
URL: https://github.com/apache/spark/pull/30296#discussion_r520260793



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala
##########
@@ -1401,6 +1401,40 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession {
     assert(e3.message.contains(errorMsg3))
   }
 
+  test("SPARK-33391: element_at with CreateArray") {

Review comment:
       It seems an overkill to have end-to-end test for it. How about we just add more tests in `CollectionExpressionsSuite.correctly handles ElementAt nullability for arrays` to test negative and invalid indices?




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

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


[GitHub] [spark] AmplabJenkins commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-723891344






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

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


[GitHub] [spark] SparkQA commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724457185


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35440/
   


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

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724465897






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

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


[GitHub] [spark] AmplabJenkins commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724525238






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

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


[GitHub] [spark] SparkQA removed a comment on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724392396


   **[Test build #130817 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130817/testReport)** for PR 30296 at commit [`4dac08d`](https://github.com/apache/spark/commit/4dac08def243c62c9edafa76cceab12847e6f750).


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

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


[GitHub] [spark] SparkQA removed a comment on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724475973


   **[Test build #130836 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130836/testReport)** for PR 30296 at commit [`fc84cac`](https://github.com/apache/spark/commit/fc84cacc8b359544352ed00efdf673eaa0a86fcc).


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

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


[GitHub] [spark] AmplabJenkins commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724534853






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

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-723915569


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/130776/
   Test FAILed.


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

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


[GitHub] [spark] leanken commented on a change in pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
leanken commented on a change in pull request #30296:
URL: https://github.com/apache/spark/pull/30296#discussion_r520262497



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala
##########
@@ -1401,6 +1401,40 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession {
     assert(e3.message.contains(errorMsg3))
   }
 
+  test("SPARK-33391: element_at with CreateArray") {

Review comment:
       sure




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

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724505787






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

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


[GitHub] [spark] SparkQA commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724534409


   **[Test build #130836 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130836/testReport)** for PR 30296 at commit [`fc84cac`](https://github.com/apache/spark/commit/fc84cacc8b359544352ed00efdf673eaa0a86fcc).
    * This patch **fails due to an unknown error code, -9**.
    * This patch merges cleanly.
    * This patch adds no public classes.


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

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


[GitHub] [spark] SparkQA commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-723914982


   **[Test build #130776 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130776/testReport)** for PR 30296 at commit [`10090a7`](https://github.com/apache/spark/commit/10090a78411788b7f4764c1d7bdf18fb27ef8247).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


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

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


[GitHub] [spark] cloud-fan commented on a change in pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30296:
URL: https://github.com/apache/spark/pull/30296#discussion_r519653740



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
##########
@@ -1966,7 +1966,7 @@ case class ElementAt(left: Expression, right: Expression)
   }
 
   override def nullable: Boolean = left.dataType match {
-    case _: ArrayType => computeNullabilityFromArray(left, right)
+    case _: ArrayType => computeNullabilityFromArray(left, right, isOneBasedIndex = true)

Review comment:
       how about `computeNullabilityFromArray(left, Subtract(right, Literal(1)))`?




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

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724142229






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

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


[GitHub] [spark] SparkQA removed a comment on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-723849875


   **[Test build #130776 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130776/testReport)** for PR 30296 at commit [`10090a7`](https://github.com/apache/spark/commit/10090a78411788b7f4764c1d7bdf18fb27ef8247).


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

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


[GitHub] [spark] SparkQA commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724392396


   **[Test build #130817 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130817/testReport)** for PR 30296 at commit [`4dac08d`](https://github.com/apache/spark/commit/4dac08def243c62c9edafa76cceab12847e6f750).


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

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724421797






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

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


[GitHub] [spark] SparkQA commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724465887


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35440/
   


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

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


[GitHub] [spark] leanken commented on a change in pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
leanken commented on a change in pull request #30296:
URL: https://github.com/apache/spark/pull/30296#discussion_r519789521



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
##########
@@ -1966,7 +1966,20 @@ case class ElementAt(left: Expression, right: Expression)
   }
 
   override def nullable: Boolean = left.dataType match {
-    case _: ArrayType => computeNullabilityFromArray(left, right)
+    case _: ArrayType =>
+      def specialNormalizeIndex: (Int, Int) => Int = {
+        (arrayLength: Int, index: Int) => {
+          if (index < 0) {
+            arrayLength + index
+          } else if (index == 0) {

Review comment:
       I am just try to follow the old behavior.




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

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


[GitHub] [spark] AmplabJenkins commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724505787






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

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


[GitHub] [spark] leanken commented on a change in pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
leanken commented on a change in pull request #30296:
URL: https://github.com/apache/spark/pull/30296#discussion_r519790144



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
##########
@@ -1966,7 +1966,20 @@ case class ElementAt(left: Expression, right: Expression)
   }
 
   override def nullable: Boolean = left.dataType match {
-    case _: ArrayType => computeNullabilityFromArray(left, right)
+    case _: ArrayType =>
+      def specialNormalizeIndex: (Int, Int) => Int = {
+        (arrayLength: Int, index: Int) => {
+          if (index < 0) {
+            arrayLength + index

Review comment:
       do we need to cover the arrayLength + index still < 0 inside this specialNormalizeIndex ?




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

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


[GitHub] [spark] leanken commented on a change in pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
leanken commented on a change in pull request #30296:
URL: https://github.com/apache/spark/pull/30296#discussion_r519789228



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
##########
@@ -1966,7 +1966,20 @@ case class ElementAt(left: Expression, right: Expression)
   }
 
   override def nullable: Boolean = left.dataType match {
-    case _: ArrayType => computeNullabilityFromArray(left, right)
+    case _: ArrayType =>
+      def specialNormalizeIndex: (Int, Int) => Int = {
+        (arrayLength: Int, index: Int) => {
+          if (index < 0) {
+            arrayLength + index
+          } else if (index == 0) {

Review comment:
       but if the passed in index is 0, it will change to -1 and call the following code. it will throw exception, but the old behavior is return a default true. 
   ```
   ar(intOrdinal).nullable
   ```




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

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


[GitHub] [spark] SparkQA commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724522678


   **[Test build #130817 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130817/testReport)** for PR 30296 at commit [`4dac08d`](https://github.com/apache/spark/commit/4dac08def243c62c9edafa76cceab12847e6f750).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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

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


[GitHub] [spark] SparkQA removed a comment on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-723978583


   **[Test build #130784 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130784/testReport)** for PR 30296 at commit [`c0bf2f2`](https://github.com/apache/spark/commit/c0bf2f20ea7fec771a7444d617afd6ca11306aa3).


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

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


[GitHub] [spark] leanken commented on a change in pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
leanken commented on a change in pull request #30296:
URL: https://github.com/apache/spark/pull/30296#discussion_r519787264



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
##########
@@ -1966,7 +1966,20 @@ case class ElementAt(left: Expression, right: Expression)
   }
 
   override def nullable: Boolean = left.dataType match {
-    case _: ArrayType => computeNullabilityFromArray(left, right)
+    case _: ArrayType =>
+      def specialNormalizeIndex: (Int, Int) => Int = {
+        (arrayLength: Int, index: Int) => {
+          if (index < 0) {
+            arrayLength + index

Review comment:
       yes, if the passing index is negative and arrayLength + index still < 0, it will still failed.




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

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


[GitHub] [spark] SparkQA commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-723874212


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35385/
   


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

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


[GitHub] [spark] SparkQA commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-723891313


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35385/
   


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

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


[GitHub] [spark] cloud-fan closed pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
cloud-fan closed pull request #30296:
URL: https://github.com/apache/spark/pull/30296


   


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

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


[GitHub] [spark] cloud-fan commented on a change in pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30296:
URL: https://github.com/apache/spark/pull/30296#discussion_r519673617



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
##########
@@ -1966,7 +1966,7 @@ case class ElementAt(left: Expression, right: Expression)
   }
 
   override def nullable: Boolean = left.dataType match {
-    case _: ArrayType => computeNullabilityFromArray(left, right)
+    case _: ArrayType => computeNullabilityFromArray(left, right, isOneBasedIndex = true)

Review comment:
       Got it. Then `isOneBasedIndex` is a misleading name. Maybe the parameter should be `normalizeIndex: (Int, Int) => Int = _._2`, which takes the array length and the index, and return the normalized 0-based non-negative index.




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

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


[GitHub] [spark] SparkQA commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-723999298


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35393/
   


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

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


[GitHub] [spark] leanken commented on a change in pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
leanken commented on a change in pull request #30296:
URL: https://github.com/apache/spark/pull/30296#discussion_r519786679



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
##########
@@ -1966,7 +1966,20 @@ case class ElementAt(left: Expression, right: Expression)
   }
 
   override def nullable: Boolean = left.dataType match {
-    case _: ArrayType => computeNullabilityFromArray(left, right)
+    case _: ArrayType =>
+      def specialNormalizeIndex: (Int, Int) => Int = {
+        (arrayLength: Int, index: Int) => {
+          if (index < 0) {
+            arrayLength + index

Review comment:
       calling nullable will not get exception or failed, if it's out of bounds, it's just returning a default true.




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

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


[GitHub] [spark] SparkQA commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-723978583


   **[Test build #130784 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130784/testReport)** for PR 30296 at commit [`c0bf2f2`](https://github.com/apache/spark/commit/c0bf2f20ea7fec771a7444d617afd6ca11306aa3).


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

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


[GitHub] [spark] AmplabJenkins commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724465897






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

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


[GitHub] [spark] cloud-fan edited a comment on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
cloud-fan edited a comment on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724515823


   GA passed, merging to master/3.0!


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

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


[GitHub] [spark] cloud-fan commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724515823


   GA passed, merging to master!


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

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


[GitHub] [spark] leanken commented on a change in pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
leanken commented on a change in pull request #30296:
URL: https://github.com/apache/spark/pull/30296#discussion_r519669776



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
##########
@@ -1966,7 +1966,7 @@ case class ElementAt(left: Expression, right: Expression)
   }
 
   override def nullable: Boolean = left.dataType match {
-    case _: ArrayType => computeNullabilityFromArray(left, right)
+    case _: ArrayType => computeNullabilityFromArray(left, right, isOneBasedIndex = true)

Review comment:
       what if it's a negative number like -1. -1 means the last one from right to left.




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

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


[GitHub] [spark] cloud-fan commented on a change in pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30296:
URL: https://github.com/apache/spark/pull/30296#discussion_r519787538



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
##########
@@ -1966,7 +1966,20 @@ case class ElementAt(left: Expression, right: Expression)
   }
 
   override def nullable: Boolean = left.dataType match {
-    case _: ArrayType => computeNullabilityFromArray(left, right)
+    case _: ArrayType =>
+      def specialNormalizeIndex: (Int, Int) => Int = {
+        (arrayLength: Int, index: Int) => {
+          if (index < 0) {
+            arrayLength + index
+          } else if (index == 0) {

Review comment:
       Actually, `ElementAt` fails at runtime if `index == 0`, so the nullable doesn't really matter.




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

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


[GitHub] [spark] itholic commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
itholic commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-728804642


   Oh, I got it. Thanks for the quick response, @leanken !! :D


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

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724506486


   Merged build finished. Test FAILed.


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

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724014853






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

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


[GitHub] [spark] SparkQA commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724505754


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35445/
   


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

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724534853


   Merged build finished. Test FAILed.


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

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


[GitHub] [spark] SparkQA commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724475973


   **[Test build #130836 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130836/testReport)** for PR 30296 at commit [`fc84cac`](https://github.com/apache/spark/commit/fc84cacc8b359544352ed00efdf673eaa0a86fcc).


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

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724534858


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/130836/
   Test FAILed.


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

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


[GitHub] [spark] AmplabJenkins commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724506486






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

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


[GitHub] [spark] leanken commented on a change in pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
leanken commented on a change in pull request #30296:
URL: https://github.com/apache/spark/pull/30296#discussion_r520297842



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala
##########
@@ -1122,11 +1122,16 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper
     val a = AttributeReference("a", IntegerType, nullable = false)()
     val b = AttributeReference("b", IntegerType, nullable = true)()
     val array = CreateArray(a :: b :: Nil)
-    assert(!ElementAt(array, Literal(0)).nullable)
-    assert(ElementAt(array, Literal(1)).nullable)
-    assert(!ElementAt(array, Subtract(Literal(2), Literal(2))).nullable)
+    assert(!ElementAt(array, Literal(1)).nullable)
+    assert(ElementAt(array, Literal(2)).nullable)
+    assert(!ElementAt(array, Subtract(Literal(2), Literal(1))).nullable)

Review comment:
       done




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

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


[GitHub] [spark] leanken commented on a change in pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
leanken commented on a change in pull request #30296:
URL: https://github.com/apache/spark/pull/30296#discussion_r520272253



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala
##########
@@ -1401,6 +1401,40 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession {
     assert(e3.message.contains(errorMsg3))
   }
 
+  test("SPARK-33391: element_at with CreateArray") {

Review comment:
       updated




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

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


[GitHub] [spark] SparkQA commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-723849875


   **[Test build #130776 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130776/testReport)** for PR 30296 at commit [`10090a7`](https://github.com/apache/spark/commit/10090a78411788b7f4764c1d7bdf18fb27ef8247).


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

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


[GitHub] [spark] SparkQA commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724506233


   **[Test build #130831 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130831/testReport)** for PR 30296 at commit [`7d09d8c`](https://github.com/apache/spark/commit/7d09d8c74a6e0b72ff7fc2ea881a3c437a4236b7).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


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

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724525238






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

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


[GitHub] [spark] AmplabJenkins commented on pull request #30296: [SPARK-33391][SQL] element_at with CreateArray not respect one based index.

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30296:
URL: https://github.com/apache/spark/pull/30296#issuecomment-724142229






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

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