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/02/09 15:29:31 UTC

[GitHub] [spark] Eric5553 opened a new pull request #27511: Refine base operator abstraction code style

Eric5553 opened a new pull request #27511: Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511
 
 
   ### What changes were proposed in this pull request?
   When doing base operator abstraction work, we found there are still some code snippet is  inconsistent with other abstraction code style. This PR addressed following two code refactor cases.
   
   **Case 1** Override keyword missed for some fields in derived classes. The compiler will not capture it if we rename some fields in the future.
   https://github.com/apache/spark/pull/27368#discussion_r376694045
   Most operators followed the practice, except:
   `EvalPythonExec` `BaseSubqueryExec` `HashJoin` `DataSourceScanExec`
   `ObjectProducerExec` `LimitExec` `WatermarkSupport` `V1FallbackWriters`
   `SupportsV1Write` `V2TableWriteExec` `WindowExecBase`
   
   **Case 2** Inconsistent abstract class filed definition. The updated style will simplify derived class definition.
   https://github.com/apache/spark/pull/27368#discussion_r375061952
   Most operators followed the practice, except:
   `EvalPythonExec` `WindowExecBase`
   
   
   ### Why are the changes needed?
   Improve the code style consistency and code quality
   
   
   ### Does this PR introduce any user-facing change?
   No
   
   
   ### How was this patch tested?
   Existing tests
   

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


With regards,
Apache Git Services

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


[GitHub] [spark] Eric5553 commented on a change in pull request #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
Eric5553 commented on a change in pull request #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#discussion_r378013963
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExecBase.scala
 ##########
 @@ -26,11 +26,12 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
 import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
 import org.apache.spark.sql.types.{CalendarIntervalType, DateType, IntegerType, TimestampType}
 
-abstract class WindowExecBase(
-    windowExpression: Seq[NamedExpression],
-    partitionSpec: Seq[Expression],
-    orderSpec: Seq[SortOrder],
-    child: SparkPlan) extends UnaryExecNode {
+abstract class WindowExecBase extends UnaryExecNode {
+  def windowExpression: Seq[NamedExpression]
+
 
 Review comment:
   Yea, thanks for the suggestion!

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584649199
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23002/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584648503
 
 
   **[Test build #118242 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118242/testReport)** for PR 27511 at commit [`9511b34`](https://github.com/apache/spark/commit/9511b349e366f85ecce97d75b3d99baedbfc4cd8).

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584797481
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-590924361
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-583904734
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/22866/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] Eric5553 commented on a change in pull request #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
Eric5553 commented on a change in pull request #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#discussion_r383776024
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonExec.scala
 ##########
 @@ -57,8 +57,9 @@ import org.apache.spark.util.Utils
  * there should be always some rows buffered in the socket or Python process, so the pulling from
  * RowQueue ALWAYS happened after pushing into it.
  */
-abstract class EvalPythonExec(udfs: Seq[PythonUDF], resultAttrs: Seq[Attribute], child: SparkPlan)
-  extends UnaryExecNode {
+abstract class EvalPythonExec extends UnaryExecNode {
 
 Review comment:
   Updated in 364f5a70d98260573f9314592ff8ca8a39459a02, 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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-590787168
 
 
   **[Test build #118919 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118919/testReport)** for PR 27511 at commit [`364f5a7`](https://github.com/apache/spark/commit/364f5a70d98260573f9314592ff8ca8a39459a02).

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


With regards,
Apache Git Services

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


[GitHub] [spark] HyukjinKwon commented on a change in pull request #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on a change in pull request #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#discussion_r383731839
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonExec.scala
 ##########
 @@ -57,8 +57,9 @@ import org.apache.spark.util.Utils
  * there should be always some rows buffered in the socket or Python process, so the pulling from
  * RowQueue ALWAYS happened after pushing into it.
  */
-abstract class EvalPythonExec(udfs: Seq[PythonUDF], resultAttrs: Seq[Attribute], child: SparkPlan)
-  extends UnaryExecNode {
+abstract class EvalPythonExec extends UnaryExecNode {
 
 Review comment:
   This one too. seems we can switch to trait.

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584786890
 
 
   **[Test build #118240 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118240/testReport)** for PR 27511 at commit [`cc709a6`](https://github.com/apache/spark/commit/cc709a64ce8582d30c22932d98b55485f6e6b284).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `case class ReusedSubqueryExec(child: BaseSubqueryExec)`
     * `case class CollectTailExec(limit: Int, child: SparkPlan) extends LimitExec `
     * `case class LocalLimitExec(limit: Int, child: SparkPlan) extends BaseLimitExec `
     * `case class GlobalLimitExec(limit: Int, child: SparkPlan) extends BaseLimitExec `

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584787950
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118240/
   Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-590924374
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23673/
   Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-583858660
 
 
   Can one of the admins verify this patch?

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584797491
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118239/
   Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-590924361
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] Eric5553 commented on a change in pull request #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
Eric5553 commented on a change in pull request #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#discussion_r383946201
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala
 ##########
 @@ -102,7 +102,7 @@ case class RowDataSourceScanExec(
     filters: Set[Filter],
     handledFilters: Set[Filter],
     rdd: RDD[InternalRow],
-    @transient relation: BaseRelation,
+    @transient override val relation: BaseRelation,
 
 Review comment:
   Oh, I think I forgot to update here. Per previous discussion, we should replace base definition to 'def' instead of adding these `override`s.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-591049585
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-590915374
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] Eric5553 commented on issue #27511: Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
Eric5553 commented on issue #27511: Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-583857889
 
 
   cc @gatorsmile @maropu 

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584788855
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118245/
   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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584787950
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118240/
   Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-591049595
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118925/
   Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-590787876
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23667/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-583935543
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27511: Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27511: Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-583857910
 
 
   Can one of the admins verify this patch?

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584644890
 
 
   **[Test build #118240 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118240/testReport)** for PR 27511 at commit [`cc709a6`](https://github.com/apache/spark/commit/cc709a64ce8582d30c22932d98b55485f6e6b284).

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584652941
 
 
   **[Test build #118245 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118245/testReport)** for PR 27511 at commit [`e8a6a4e`](https://github.com/apache/spark/commit/e8a6a4ea2d8db26da27dfe371b4a355647bc9b15).

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-583904582
 
 
   **[Test build #118101 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118101/testReport)** for PR 27511 at commit [`87c2a32`](https://github.com/apache/spark/commit/87c2a32d84fe42deca43ab006d8978f4c52839c2).

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-590787861
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-590787876
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23667/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-583904582
 
 
   **[Test build #118101 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118101/testReport)** for PR 27511 at commit [`87c2a32`](https://github.com/apache/spark/commit/87c2a32d84fe42deca43ab006d8978f4c52839c2).

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-591048725
 
 
   **[Test build #118925 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118925/testReport)** for PR 27511 at commit [`839f7b0`](https://github.com/apache/spark/commit/839f7b04a766252d034e46fea6b2da3c5ed3c361).
    * 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584819119
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584993751
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23031/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584993352
 
 
   **[Test build #118272 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118272/testReport)** for PR 27511 at commit [`d07cc1b`](https://github.com/apache/spark/commit/d07cc1b5f279e06716771f076a661f3d0b0e1ab4).

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-590914546
 
 
   **[Test build #118919 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118919/testReport)** for PR 27511 at commit [`364f5a7`](https://github.com/apache/spark/commit/364f5a70d98260573f9314592ff8ca8a39459a02).
    * This patch **fails SparkR unit tests**.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `trait EvalPythonExec extends UnaryExecNode `
     * `trait WindowExecBase extends UnaryExecNode `

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584642076
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/22999/
   Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584819128
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118242/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584645500
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584993744
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584645515
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23000/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584797491
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118239/
   Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584652941
 
 
   **[Test build #118245 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118245/testReport)** for PR 27511 at commit [`e8a6a4e`](https://github.com/apache/spark/commit/e8a6a4ea2d8db26da27dfe371b4a355647bc9b15).

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584819119
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584644890
 
 
   **[Test build #118240 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118240/testReport)** for PR 27511 at commit [`cc709a6`](https://github.com/apache/spark/commit/cc709a64ce8582d30c22932d98b55485f6e6b284).

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584788849
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] HyukjinKwon closed pull request #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
HyukjinKwon closed pull request #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511
 
 
   

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584993751
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23031/
   Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-583857910
 
 
   Can one of the admins verify this patch?

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-590915379
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118919/
   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


With regards,
Apache Git Services

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


[GitHub] [spark] Eric5553 commented on a change in pull request #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
Eric5553 commented on a change in pull request #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#discussion_r383776024
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonExec.scala
 ##########
 @@ -57,8 +57,9 @@ import org.apache.spark.util.Utils
  * there should be always some rows buffered in the socket or Python process, so the pulling from
  * RowQueue ALWAYS happened after pushing into it.
  */
-abstract class EvalPythonExec(udfs: Seq[PythonUDF], resultAttrs: Seq[Attribute], child: SparkPlan)
-  extends UnaryExecNode {
+abstract class EvalPythonExec extends UnaryExecNode {
 
 Review comment:
   Updated in 364f5a70d98260573f9314592ff8ca8a39459a02

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584817969
 
 
   **[Test build #118242 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118242/testReport)** for PR 27511 at commit [`9511b34`](https://github.com/apache/spark/commit/9511b349e366f85ecce97d75b3d99baedbfc4cd8).
    * 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584993352
 
 
   **[Test build #118272 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118272/testReport)** for PR 27511 at commit [`d07cc1b`](https://github.com/apache/spark/commit/d07cc1b5f279e06716771f076a661f3d0b0e1ab4).

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-583904734
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/22866/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584819128
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118242/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584993744
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#discussion_r377017452
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/python/WindowInPandasExec.scala
 ##########
 @@ -80,11 +80,11 @@ import org.apache.spark.util.Utils
  * window.
  */
 case class WindowInPandasExec(
-    windowExpression: Seq[NamedExpression],
-    partitionSpec: Seq[Expression],
-    orderSpec: Seq[SortOrder],
+    override val windowExpression: Seq[NamedExpression],
+    override val partitionSpec: Seq[Expression],
 
 Review comment:
   same question. `case class` means everything is `val` so this is just adding `override`.

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


With regards,
Apache Git Services

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


[GitHub] [spark] maropu commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
maropu commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-583904240
 
 
   ok to 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


With regards,
Apache Git Services

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


[GitHub] [spark] HyukjinKwon commented on a change in pull request #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on a change in pull request #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#discussion_r376955135
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/python/WindowInPandasExec.scala
 ##########
 @@ -80,11 +80,11 @@ import org.apache.spark.util.Utils
  * window.
  */
 case class WindowInPandasExec(
-    windowExpression: Seq[NamedExpression],
-    partitionSpec: Seq[Expression],
-    orderSpec: Seq[SortOrder],
+    override val windowExpression: Seq[NamedExpression],
+    override val partitionSpec: Seq[Expression],
 
 Review comment:
   Is `override val` required here?

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-585069339
 
 
   **[Test build #118272 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118272/testReport)** for PR 27511 at commit [`d07cc1b`](https://github.com/apache/spark/commit/d07cc1b5f279e06716771f076a661f3d0b0e1ab4).
    * 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-590787861
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] Eric5553 commented on a change in pull request #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
Eric5553 commented on a change in pull request #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#discussion_r377657438
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/python/WindowInPandasExec.scala
 ##########
 @@ -80,11 +80,11 @@ import org.apache.spark.util.Utils
  * window.
  */
 case class WindowInPandasExec(
-    windowExpression: Seq[NamedExpression],
-    partitionSpec: Seq[Expression],
-    orderSpec: Seq[SortOrder],
+    override val windowExpression: Seq[NamedExpression],
+    override val partitionSpec: Seq[Expression],
 
 Review comment:
   @HyukjinKwon @cloud-fan  Thanks! I dropped most changes based on this comment https://github.com/apache/spark/pull/27368#discussion_r376925299. So current refactor rules are:
   
   1. If abstract class or trait defined with `def`, then no override needed in derived class.
   2. Change `HashJoin` definition to `def` to avoid `override` in derived classes.
   3. Remove abstract class constructer part of `EvalPythonExec` `WindowExecBase` to sync with other operator code style. 

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584797481
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-590923618
 
 
   **[Test build #118925 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118925/testReport)** for PR 27511 at commit [`839f7b0`](https://github.com/apache/spark/commit/839f7b04a766252d034e46fea6b2da3c5ed3c361).

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-591049595
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118925/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584641384
 
 
   **[Test build #118239 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118239/testReport)** for PR 27511 at commit [`7ad8d83`](https://github.com/apache/spark/commit/7ad8d83b227f7b41b76502c06f82969e594b58d8).

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584642062
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#discussion_r383819544
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala
 ##########
 @@ -158,7 +158,7 @@ case class RowDataSourceScanExec(
  * @param tableIdentifier identifier for the table in the metastore.
  */
 case class FileSourceScanExec(
-    @transient relation: HadoopFsRelation,
+    @transient override val relation: HadoopFsRelation,
 
 Review comment:
   ditto

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-583858660
 
 
   Can one of the admins verify this patch?

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584645500
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584648503
 
 
   **[Test build #118242 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118242/testReport)** for PR 27511 at commit [`9511b34`](https://github.com/apache/spark/commit/9511b349e366f85ecce97d75b3d99baedbfc4cd8).

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


With regards,
Apache Git Services

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


[GitHub] [spark] HyukjinKwon commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-591827725
 
 
   Merged 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


With regards,
Apache Git Services

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


[GitHub] [spark] HyukjinKwon commented on a change in pull request #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on a change in pull request #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#discussion_r383731277
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExecBase.scala
 ##########
 @@ -26,11 +26,10 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
 import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
 import org.apache.spark.sql.types.{CalendarIntervalType, DateType, IntegerType, TimestampType}
 
-abstract class WindowExecBase(
-    windowExpression: Seq[NamedExpression],
-    partitionSpec: Seq[Expression],
-    orderSpec: Seq[SortOrder],
-    child: SparkPlan) extends UnaryExecNode {
+abstract class WindowExecBase extends UnaryExecNode {
 
 Review comment:
   I think we can switch it to `trait` now.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-585069964
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118272/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-585069956
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-590787168
 
 
   **[Test build #118919 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118919/testReport)** for PR 27511 at commit [`364f5a7`](https://github.com/apache/spark/commit/364f5a70d98260573f9314592ff8ca8a39459a02).

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


With regards,
Apache Git Services

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


[GitHub] [spark] maropu edited a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
maropu edited a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-583904256
 
 
   cc: @cloud-fan @dongjoon-hyun 

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584649199
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23002/
   Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-583904730
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584641384
 
 
   **[Test build #118239 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118239/testReport)** for PR 27511 at commit [`7ad8d83`](https://github.com/apache/spark/commit/7ad8d83b227f7b41b76502c06f82969e594b58d8).

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584649190
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] Eric5553 commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
Eric5553 commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-591828655
 
 
   Thanks a lot! @HyukjinKwon @cloud-fan @maropu 

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584787935
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] maropu commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
maropu commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584943993
 
 
   Looks fine now and I'll leave this to the others. @gatorsmile @cloud-fan @HyukjinKwon 

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584649190
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584787935
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-590915379
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118919/
   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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-583935548
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118101/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-590924374
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23673/
   Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-583935548
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118101/
   Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-583935543
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] Eric5553 commented on a change in pull request #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
Eric5553 commented on a change in pull request #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#discussion_r383775938
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExecBase.scala
 ##########
 @@ -26,11 +26,10 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
 import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
 import org.apache.spark.sql.types.{CalendarIntervalType, DateType, IntegerType, TimestampType}
 
-abstract class WindowExecBase(
-    windowExpression: Seq[NamedExpression],
-    partitionSpec: Seq[Expression],
-    orderSpec: Seq[SortOrder],
-    child: SparkPlan) extends UnaryExecNode {
+abstract class WindowExecBase extends UnaryExecNode {
 
 Review comment:
   I see, updated in 364f5a70d98260573f9314592ff8ca8a39459a02.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#discussion_r377980896
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExecBase.scala
 ##########
 @@ -26,11 +26,12 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
 import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
 import org.apache.spark.sql.types.{CalendarIntervalType, DateType, IntegerType, TimestampType}
 
-abstract class WindowExecBase(
-    windowExpression: Seq[NamedExpression],
-    partitionSpec: Seq[Expression],
-    orderSpec: Seq[SortOrder],
-    child: SparkPlan) extends UnaryExecNode {
+abstract class WindowExecBase extends UnaryExecNode {
+  def windowExpression: Seq[NamedExpression]
+
 
 Review comment:
   super nit: I feel its better to avoid this blank line along with the others: https://github.com/apache/spark/pull/27368/files#diff-193703359b7f77e83814f1dece93be5fR28-R31

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-590923618
 
 
   **[Test build #118925 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118925/testReport)** for PR 27511 at commit [`839f7b0`](https://github.com/apache/spark/commit/839f7b04a766252d034e46fea6b2da3c5ed3c361).

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-585069964
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118272/
   Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-585069956
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584788123
 
 
   **[Test build #118245 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118245/testReport)** for PR 27511 at commit [`e8a6a4e`](https://github.com/apache/spark/commit/e8a6a4ea2d8db26da27dfe371b4a355647bc9b15).
    * This patch **fails PySpark 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584645515
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23000/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584796476
 
 
   **[Test build #118239 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118239/testReport)** for PR 27511 at commit [`7ad8d83`](https://github.com/apache/spark/commit/7ad8d83b227f7b41b76502c06f82969e594b58d8).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `case class SubqueryExec(name: String, child: SparkPlan)`
     * `case class CollectLimitExec(limit: Int, child: SparkPlan) extends LimitExec `
     * `case class ArrowEvalPythonExec(udfs: Seq[PythonUDF], resultAttrs: Seq[Attribute], child: SparkPlan,`
     * `case class BatchEvalPythonExec(udfs: Seq[PythonUDF], resultAttrs: Seq[Attribute], child: SparkPlan)`
     * `case class StreamingLocalLimitExec(limit: Int, child: SparkPlan)`

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584788855
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118245/
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#discussion_r383819497
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala
 ##########
 @@ -102,7 +102,7 @@ case class RowDataSourceScanExec(
     filters: Set[Filter],
     handledFilters: Set[Filter],
     rdd: RDD[InternalRow],
-    @transient relation: BaseRelation,
+    @transient override val relation: BaseRelation,
 
 Review comment:
   do we need `override`?

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


With regards,
Apache Git Services

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


[GitHub] [spark] maropu commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
maropu commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-583904256
 
 
   cc: @cloud-fan

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-583935214
 
 
   **[Test build #118101 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118101/testReport)** for PR 27511 at commit [`87c2a32`](https://github.com/apache/spark/commit/87c2a32d84fe42deca43ab006d8978f4c52839c2).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `case class SubqueryExec(override val name: String, override val child: SparkPlan)`
     * `case class ReusedSubqueryExec(override val child: BaseSubqueryExec)`
     * `case class CollectLimitExec(override val limit: Int, child: SparkPlan) extends LimitExec `
     * `case class CollectTailExec(override val limit: Int, child: SparkPlan) extends LimitExec `
     * `case class LocalLimitExec(override val limit: Int, child: SparkPlan) extends BaseLimitExec `
     * `case class GlobalLimitExec(override val limit: Int, child: SparkPlan) extends BaseLimitExec `
     * `case class ArrowEvalPythonExec(`
     * `case class BatchEvalPythonExec(`
     * `abstract class EvalPythonExec extends UnaryExecNode `
     * `case class StreamingLocalLimitExec(override val limit: Int, child: SparkPlan)`
     * `abstract class WindowExecBase extends UnaryExecNode `

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


With regards,
Apache Git Services

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


[GitHub] [spark] Eric5553 commented on a change in pull request #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
Eric5553 commented on a change in pull request #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#discussion_r383775938
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExecBase.scala
 ##########
 @@ -26,11 +26,10 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
 import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
 import org.apache.spark.sql.types.{CalendarIntervalType, DateType, IntegerType, TimestampType}
 
-abstract class WindowExecBase(
-    windowExpression: Seq[NamedExpression],
-    partitionSpec: Seq[Expression],
-    orderSpec: Seq[SortOrder],
-    child: SparkPlan) extends UnaryExecNode {
+abstract class WindowExecBase extends UnaryExecNode {
 
 Review comment:
   I see, updated in 364f5a70d98260573f9314592ff8ca8a39459a02. 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584788849
 
 
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-590915374
 
 
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-591049585
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-583904730
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584642076
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/22999/
   Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27511: [SPARK-30765][SQL] Refine base operator abstraction code style
URL: https://github.com/apache/spark/pull/27511#issuecomment-584642062
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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