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 2021/01/17 00:09:03 UTC

[GitHub] [spark] tanelk opened a new pull request #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

tanelk opened a new pull request #31213:
URL: https://github.com/apache/spark/pull/31213


   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
     2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
     4. Be sure to keep the PR description updated to reflect all changes.
     5. Please write your PR title to summarize what this PR proposes.
     6. If possible, provide a concise example to reproduce the issue for a faster review.
     7. If you want to add a new configuration, please read the guideline first for naming configurations in
        'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
   -->
   
   ### What changes were proposed in this pull request?
   <!--
   Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   Rewrote one `ExtractGenerator` case such that it would have no side effect.
   
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   With the dataframe api it is possible to have a lazy sequence as the `output` of a `LogicalPlan`. When exploding a column on this dataframe using the `withColumn("newName", explode(col("name")))` method, the `ExtractGenerator` does not extract the generator and `CheckAnalysis` would throw an exception.
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   Bugfix
   Before this, the work around was to put `.select("*")` before the explode.
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   -->
   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] srowen commented on pull request #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


   Merged to master. I'm satisfied with the motivation and test.


----------------------------------------------------------------
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 #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


   **[Test build #134154 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/134154/testReport)** for PR 31213 at commit [`08e367c`](https://github.com/apache/spark/commit/08e367c37d717fa7ad662b49afb0b77ea1de5d19).


----------------------------------------------------------------
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] srowen commented on pull request #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


   Hm, tough call. Can you invoke the second constructor by adding `:_*` to the end of the argument? But then I suppose that's not testing what you want to test either. Can you pass a Scala Stream here maybe? not sure if that works. It may work to create a new test-suite in a 2.12-only src/test directory if that's not too much trouble. It sounds like this can't happen in 2.13 anyway so the test isn't relevant.


----------------------------------------------------------------
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 #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


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


----------------------------------------------------------------
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 #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


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


----------------------------------------------------------------
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] LuciferYang commented on pull request #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


   @tanelk In Scala 2.13, `Views`  are collections whose transformation operations are non strict and they no longer extend their corresponding collection type, for example, an `IndexedSeqView` no longer extends `IndexedSeq`.
   
   Why do we need a non strict collection here? Can it be a strict collection? 
   If it must be a non strict collection, may need to define the type of `output` as `scala.collection.SeqView` for version compatibility.
   


----------------------------------------------------------------
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 #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


   **[Test build #134154 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/134154/testReport)** for PR 31213 at commit [`08e367c`](https://github.com/apache/spark/commit/08e367c37d717fa7ad662b49afb0b77ea1de5d19).


----------------------------------------------------------------
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 #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


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


----------------------------------------------------------------
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] tanelk commented on pull request #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


   @LuciferYang, you have been working with 2.13. 
   Any recommendations on how to get a lazy Seq on both 2.12 and 2.13.
   On 2.13 the Seq.view does not return a Seq anymore.


----------------------------------------------------------------
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] srowen commented on a change in pull request #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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



##########
File path: sql/catalyst/src/test/scala-2.13/org/apache/spark/sql/catalyst/analysis/ExtractGeneratorSuite.scala
##########
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import scala.collection.immutable.LazyList
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.types._
+
+class ExtractGeneratorSuite extends AnalysisTest {

Review comment:
       I'd add comments in each test file noting that there is a parallel one in the other source tree, so people realize that both need to change.




----------------------------------------------------------------
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] srowen closed pull request #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

Posted by GitBox <gi...@apache.org>.
srowen closed pull request #31213:
URL: https://github.com/apache/spark/pull/31213


   


----------------------------------------------------------------
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] srowen commented on a change in pull request #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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



##########
File path: sql/catalyst/src/test/scala-2.13/org/apache/spark/sql/catalyst/analysis/ExtractGeneratorSuite.scala
##########
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import scala.collection.immutable.LazyList
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.types._
+
+class ExtractGeneratorSuite extends AnalysisTest {

Review comment:
       If you'll add the comments I'll merge it @tanelk 




----------------------------------------------------------------
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 #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


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


----------------------------------------------------------------
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 #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


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


----------------------------------------------------------------
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] maropu commented on a change in pull request #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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



##########
File path: sql/catalyst/src/test/scala-2.12/org/apache/spark/sql/catalyst/analysis/ExtractGeneratorSuite.scala
##########
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.types._
+
+/**
+ * Note: this test supports Scala 2.12. A parallel source tree has a 2.13 implementation.
+ */
+class ExtractGeneratorSuite extends AnalysisTest {
+
+  test("SPARK-34141: ExtractGenerator with lazy project list") {
+    val b = AttributeReference("b", ArrayType(StringType))()
+
+    val columns = AttributeReference("a", StringType)() :: b :: Nil
+    val explode = Alias(Explode(b), "c")()
+
+    // view is a lazy seq
+    val rel = LocalRelation(output = columns.view)

Review comment:
       Is it possible for end users to get stuck on this issue? If possible, could you add end-2-end tests, too?




----------------------------------------------------------------
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 #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


   **[Test build #134158 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/134158/testReport)** for PR 31213 at commit [`cefc79f`](https://github.com/apache/spark/commit/cefc79fb4f1d18b64858e95143209bcb884916ca).
    * 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] tanelk commented on pull request #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


   @srowen 
   There is a correctness issue, when an lazy `Seq` is given as the project list (for example `SeqView`).
   I did fix it, but the unit test does not compile on 2.13, because there `SeqView` does not implement `Seq` any more.
   
   Would it be okay do not add the UT or is there some lazy `Seq` present on both 2.12 and 2.13.


----------------------------------------------------------------
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 #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


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


----------------------------------------------------------------
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 #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


   **[Test build #134158 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/134158/testReport)** for PR 31213 at commit [`cefc79f`](https://github.com/apache/spark/commit/cefc79fb4f1d18b64858e95143209bcb884916ca).


----------------------------------------------------------------
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 #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


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


----------------------------------------------------------------
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 #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


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


----------------------------------------------------------------
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 #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


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


----------------------------------------------------------------
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 #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


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


----------------------------------------------------------------
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 #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


   **[Test build #134689 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/134689/testReport)** for PR 31213 at commit [`82ca8f5`](https://github.com/apache/spark/commit/82ca8f528eae0ce960f26fc6fd5a10601afa3723).
    * 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] AmplabJenkins commented on pull request #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


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


----------------------------------------------------------------
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 #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


   **[Test build #134511 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/134511/testReport)** for PR 31213 at commit [`8dd6da0`](https://github.com/apache/spark/commit/8dd6da0f4dcd2b2ddbf5e3d2dbc5b91010fcbe89).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `class ExtractGeneratorSuite extends AnalysisTest `
     * `class ExtractGeneratorSuite extends AnalysisTest `


----------------------------------------------------------------
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 #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


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


----------------------------------------------------------------
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 #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


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


----------------------------------------------------------------
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 #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


   **[Test build #134689 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/134689/testReport)** for PR 31213 at commit [`82ca8f5`](https://github.com/apache/spark/commit/82ca8f528eae0ce960f26fc6fd5a10601afa3723).


----------------------------------------------------------------
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 #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


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


----------------------------------------------------------------
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 #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


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


----------------------------------------------------------------
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 #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


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


----------------------------------------------------------------
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 #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


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


----------------------------------------------------------------
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] tanelk commented on pull request #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


   Well that is the test case I'm trying to cover.
   in 2.12 the input can be lazy Seq, but the current code breaks then.
   Perhaps it's enough to say that I manually tested it and not add the 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] LuciferYang commented on pull request #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


   > Well that is the test case I'm trying to cover.
   in 2.12 the input can be lazy Seq, but the current code breaks then.
   Perhaps it's enough to say that I manually tested it and not add the UT.
   
   New UT in this pr will break the compilation of Scala 2.13,  @tanelk want to get a lazy `Seq` on both 2.12 and 2.13, but `SeqView` is not `Seq` in Scala 2.13,  do you have any suggestions ? @srowen 


----------------------------------------------------------------
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 #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


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


----------------------------------------------------------------
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] LuciferYang commented on pull request #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


   cc @srowen 


----------------------------------------------------------------
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 #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


   **[Test build #134154 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/134154/testReport)** for PR 31213 at commit [`08e367c`](https://github.com/apache/spark/commit/08e367c37d717fa7ad662b49afb0b77ea1de5d19).
    * 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] AmplabJenkins removed a comment on pull request #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


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


----------------------------------------------------------------
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] LuciferYang commented on pull request #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


   ![image](https://user-images.githubusercontent.com/1475305/105042835-b0f3ce00-5a9f-11eb-9257-2682bfbca50b.png)
   


----------------------------------------------------------------
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 #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


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


----------------------------------------------------------------
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] tanelk commented on pull request #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


   @HyukjinKwon , any chance that this bugfix could get to 3.1.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] SparkQA commented on pull request #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


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


----------------------------------------------------------------
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] srowen commented on pull request #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


   Hm why does Seq vs SeqView matter here? what's the compile error?


----------------------------------------------------------------
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] tanelk commented on pull request #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


   I went ahead and added version dependent tests. Thanks for the advice.


----------------------------------------------------------------
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 #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


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


----------------------------------------------------------------
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 #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


   **[Test build #134511 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/134511/testReport)** for PR 31213 at commit [`8dd6da0`](https://github.com/apache/spark/commit/8dd6da0f4dcd2b2ddbf5e3d2dbc5b91010fcbe89).


----------------------------------------------------------------
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 #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


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


----------------------------------------------------------------
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] HyukjinKwon commented on pull request #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


   Oh, just saw it. LGTM


-- 
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 #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


   **[Test build #134158 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/134158/testReport)** for PR 31213 at commit [`cefc79f`](https://github.com/apache/spark/commit/cefc79fb4f1d18b64858e95143209bcb884916ca).


----------------------------------------------------------------
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 #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


   **[Test build #134689 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/134689/testReport)** for PR 31213 at commit [`82ca8f5`](https://github.com/apache/spark/commit/82ca8f528eae0ce960f26fc6fd5a10601afa3723).


----------------------------------------------------------------
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] maropu commented on a change in pull request #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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



##########
File path: sql/catalyst/src/test/scala-2.12/org/apache/spark/sql/catalyst/analysis/ExtractGeneratorSuite.scala
##########
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.types._
+
+/**
+ * Note: this test supports Scala 2.12. A parallel source tree has a 2.13 implementation.
+ */
+class ExtractGeneratorSuite extends AnalysisTest {
+
+  test("SPARK-34141: ExtractGenerator with lazy project list") {
+    val b = AttributeReference("b", ArrayType(StringType))()
+
+    val columns = AttributeReference("a", StringType)() :: b :: Nil
+    val explode = Alias(Explode(b), "c")()
+
+    // view is a lazy seq
+    val rel = LocalRelation(output = columns.view)

Review comment:
       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 #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


   **[Test build #134511 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/134511/testReport)** for PR 31213 at commit [`8dd6da0`](https://github.com/apache/spark/commit/8dd6da0f4dcd2b2ddbf5e3d2dbc5b91010fcbe89).


----------------------------------------------------------------
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 #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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


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


----------------------------------------------------------------
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] tanelk commented on a change in pull request #31213: [SPARK-34141][SQL] Remove side effect from ExtractGenerator

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



##########
File path: sql/catalyst/src/test/scala-2.12/org/apache/spark/sql/catalyst/analysis/ExtractGeneratorSuite.scala
##########
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.types._
+
+/**
+ * Note: this test supports Scala 2.12. A parallel source tree has a 2.13 implementation.
+ */
+class ExtractGeneratorSuite extends AnalysisTest {
+
+  test("SPARK-34141: ExtractGenerator with lazy project list") {
+    val b = AttributeReference("b", ArrayType(StringType))()
+
+    val columns = AttributeReference("a", StringType)() :: b :: Nil
+    val explode = Alias(Explode(b), "c")()
+
+    // view is a lazy seq
+    val rel = LocalRelation(output = columns.view)

Review comment:
       I hit this issue while using spark in java. The step before was an join, where I used `JavaConverters.collectionAsScalaIterable(Arrays.asList(columns)).toSeq()` as the join condition. The JavaConverters helper returns an lazy collection.
   I can take a look at on how to e2e test this.




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