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

[GitHub] [spark] HyukjinKwon opened a new pull request #35626: [SPARK-38240][SQL][FOLLOW-UP] Make RuntimeReplaceableAggregate as an add-on of RuntimeReplaceable

HyukjinKwon opened a new pull request #35626:
URL: https://github.com/apache/spark/pull/35626


   ### What changes were proposed in this pull request?
   
   This PR is a followup of https://github.com/apache/spark/pull/35534 (https://github.com/apache/spark/pull/35534#discussion_r809708015) that proposes to make `RuntimeReplaceableAggregate` as an add-on of `RuntimeReplaceable`. Now, `RuntimeReplaceable` has two add-ons that can be mixed-in:
   - `RuntimeReplaceableAggregate` for aggregate functions
   - `InheritAnalysisRules` to inherit analysis rules
   
   ### Why are the changes needed?
   
   To make the hierarchy of class similar.
   
   ### Does this PR introduce _any_ user-facing change?
   
   No, dev-only.
   
   ### How was this patch tested?
   
   I tested that it compiles fine. CI in this PR will test it out.
   


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

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

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



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


[GitHub] [spark] cloud-fan commented on a change in pull request #35626: [SPARK-38240][SQL][FOLLOW-UP] Make RuntimeReplaceableAggregate as an add-on of AggregateFunction

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CountIf.scala
##########
@@ -34,8 +34,12 @@ import org.apache.spark.sql.types.{AbstractDataType, BooleanType}
   """,
   group = "agg_funcs",
   since = "3.0.0")
-case class CountIf(child: Expression) extends RuntimeReplaceableAggregate
-  with ImplicitCastInputTypes with UnaryLike[Expression] {
+case class CountIf(
+    child: Expression)

Review comment:
       ```suggestion
   case class CountIf(child: Expression)
   ```




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

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

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



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


[GitHub] [spark] HyukjinKwon closed pull request #35626: [SPARK-38240][SQL][FOLLOW-UP] Make RuntimeReplaceableAggregate as an add-on of AggregateFunction

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


   


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

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

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



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


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #35626: [SPARK-38240][SQL][FOLLOW-UP] Make RuntimeReplaceableAggregate as an add-on of RuntimeReplaceable

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #35626:
URL: https://github.com/apache/spark/pull/35626#discussion_r813203156



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CountIf.scala
##########
@@ -34,8 +34,11 @@ import org.apache.spark.sql.types.{AbstractDataType, BooleanType}
   """,
   group = "agg_funcs",
   since = "3.0.0")
-case class CountIf(child: Expression) extends RuntimeReplaceableAggregate
-  with ImplicitCastInputTypes with UnaryLike[Expression] {
+case class CountIf(child: Expression)
+    extends ReplaceableAggregateFunction
+    with RuntimeReplaceable
+    with ImplicitCastInputTypes
+    with UnaryLike[Expression] {

Review comment:
       Ur, is the internal Databricks Scala style guide changed, @HyukjinKwon ? In the community, it's still `2-space` for `extends` and `with`.
   - https://github.com/databricks/scala-style-guide

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/RegrCount.scala
##########
@@ -37,7 +37,10 @@ import org.apache.spark.sql.types.{AbstractDataType, NumericType}
   group = "agg_funcs",
   since = "3.3.0")
 case class RegrCount(left: Expression, right: Expression)
-  extends RuntimeReplaceableAggregate with ImplicitCastInputTypes with BinaryLike[Expression] {
+    extends ReplaceableAggregateFunction
+    with RuntimeReplaceable
+    with ImplicitCastInputTypes
+    with BinaryLike[Expression] {

Review comment:
       ditto.

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/boolAggregates.scala
##########
@@ -34,8 +34,10 @@ import org.apache.spark.sql.types._
   """,
   group = "agg_funcs",
   since = "3.0.0")
-case class BoolAnd(child: Expression) extends RuntimeReplaceableAggregate
-  with ImplicitCastInputTypes with UnaryLike[Expression] {
+case class BoolAnd(child: Expression)
+    extends ReplaceableAggregateFunction
+    with RuntimeReplaceable
+    with ImplicitCastInputTypes with UnaryLike[Expression] {

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.

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

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



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


[GitHub] [spark] HyukjinKwon commented on a change in pull request #35626: [SPARK-38240][SQL][FOLLOW-UP] Make RuntimeReplaceableAggregate as an add-on of RuntimeReplaceable

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CountIf.scala
##########
@@ -34,8 +34,11 @@ import org.apache.spark.sql.types.{AbstractDataType, BooleanType}
   """,
   group = "agg_funcs",
   since = "3.0.0")
-case class CountIf(child: Expression) extends RuntimeReplaceableAggregate
-  with ImplicitCastInputTypes with UnaryLike[Expression] {
+case class CountIf(child: Expression)
+    extends RuntimeReplaceable
+    with ReplaceableAggregateFunction

Review comment:
       I checked runtime replaceable aggregate functions. Seems like none requires `InheritAnalysisRules` for 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.

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

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



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


[GitHub] [spark] HyukjinKwon commented on a change in pull request #35626: [SPARK-38240][SQL][FOLLOW-UP] Make RuntimeReplaceableAggregate as an add-on of RuntimeReplaceable

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CountIf.scala
##########
@@ -34,8 +34,11 @@ import org.apache.spark.sql.types.{AbstractDataType, BooleanType}
   """,
   group = "agg_funcs",
   since = "3.0.0")
-case class CountIf(child: Expression) extends RuntimeReplaceableAggregate
-  with ImplicitCastInputTypes with UnaryLike[Expression] {
+case class CountIf(child: Expression)
+    extends ReplaceableAggregateFunction
+    with RuntimeReplaceable
+    with ImplicitCastInputTypes
+    with UnaryLike[Expression] {

Review comment:
       oh yeah. my bad :-).




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

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

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



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


[GitHub] [spark] HyukjinKwon commented on a change in pull request #35626: [SPARK-38240][SQL][FOLLOW-UP] Make RuntimeReplaceableAggregate as an add-on of RuntimeReplaceable

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
##########
@@ -390,18 +390,18 @@ trait InheritAnalysisRules extends UnaryLike[Expression] { self: RuntimeReplacea
 }
 
 /**
- * An aggregate expression that gets rewritten (currently by the optimizer) into a
+ * An add-on of [[RuntimeReplaceable]]. This gets rewritten (currently by the optimizer) into a
  * different aggregate expression for evaluation. This is mainly used to provide compatibility
  * with other databases. For example, we use this to support every, any/some aggregates by rewriting
  * them with Min and Max respectively.
  */
-abstract class RuntimeReplaceableAggregate extends AggregateFunction with RuntimeReplaceable {
-  def aggBufferSchema: StructType = throw new IllegalStateException(
-    "RuntimeReplaceableAggregate.aggBufferSchema should not be called")
-  def aggBufferAttributes: Seq[AttributeReference] = throw new IllegalStateException(
-    "RuntimeReplaceableAggregate.aggBufferAttributes should not be called")
-  def inputAggBufferAttributes: Seq[AttributeReference] = throw new IllegalStateException(
-    "RuntimeReplaceableAggregate.inputAggBufferAttributes should not be called")
+trait ReplaceableAggregateFunction extends AggregateFunction { self: RuntimeReplaceable =>

Review comment:
       Sure




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

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

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



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


[GitHub] [spark] cloud-fan commented on a change in pull request #35626: [SPARK-38240][SQL][FOLLOW-UP] Make RuntimeReplaceableAggregate as an add-on of AggregateFunction

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/linearRegression.scala
##########
@@ -65,8 +68,13 @@ case class RegrCount(left: Expression, right: Expression)
   group = "agg_funcs",
   since = "3.3.0")
 // scalastyle:on line.size.limit
-case class RegrAvgX(left: Expression, right: Expression)
-  extends RuntimeReplaceableAggregate with ImplicitCastInputTypes with BinaryLike[Expression] {
+case class RegrAvgX(
+    left: Expression,
+    right: Expression)

Review comment:
       can you fix these two as well? Ideally the constructor should be in one line if possible, and `extends`, `with` can be in new lines.




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

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

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



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


[GitHub] [spark] HyukjinKwon commented on a change in pull request #35626: [SPARK-38240][SQL][FOLLOW-UP] Make RuntimeReplaceableAggregate as an add-on of AggregateFunction

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/linearRegression.scala
##########
@@ -65,8 +68,13 @@ case class RegrCount(left: Expression, right: Expression)
   group = "agg_funcs",
   since = "3.3.0")
 // scalastyle:on line.size.limit
-case class RegrAvgX(left: Expression, right: Expression)
-  extends RuntimeReplaceableAggregate with ImplicitCastInputTypes with BinaryLike[Expression] {
+case class RegrAvgX(
+    left: Expression,
+    right: Expression)

Review comment:
       This is actually following DB style guidelines:
   
   > For classes whose header doesn't fit in two lines, use 4 space indentation for its parameters, put each in each line, put the extends on the next line with 2 space indent, and add a blank line after class header.
   
   https://github.com/databricks/scala-style-guide#spacing-and-indentation

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/linearRegression.scala
##########
@@ -65,8 +68,13 @@ case class RegrCount(left: Expression, right: Expression)
   group = "agg_funcs",
   since = "3.3.0")
 // scalastyle:on line.size.limit
-case class RegrAvgX(left: Expression, right: Expression)
-  extends RuntimeReplaceableAggregate with ImplicitCastInputTypes with BinaryLike[Expression] {
+case class RegrAvgX(
+    left: Expression,
+    right: Expression)

Review comment:
       This is actually following the style guidelines:
   
   > For classes whose header doesn't fit in two lines, use 4 space indentation for its parameters, put each in each line, put the extends on the next line with 2 space indent, and add a blank line after class header.
   
   https://github.com/databricks/scala-style-guide#spacing-and-indentation




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

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

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



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


[GitHub] [spark] cloud-fan commented on a change in pull request #35626: [SPARK-38240][SQL][FOLLOW-UP] Make RuntimeReplaceableAggregate as an add-on of RuntimeReplaceable

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
##########
@@ -390,18 +390,18 @@ trait InheritAnalysisRules extends UnaryLike[Expression] { self: RuntimeReplacea
 }
 
 /**
- * An aggregate expression that gets rewritten (currently by the optimizer) into a
+ * An add-on of [[RuntimeReplaceable]]. This gets rewritten (currently by the optimizer) into a
  * different aggregate expression for evaluation. This is mainly used to provide compatibility
  * with other databases. For example, we use this to support every, any/some aggregates by rewriting
  * them with Min and Max respectively.
  */
-abstract class RuntimeReplaceableAggregate extends AggregateFunction with RuntimeReplaceable {
-  def aggBufferSchema: StructType = throw new IllegalStateException(
-    "RuntimeReplaceableAggregate.aggBufferSchema should not be called")
-  def aggBufferAttributes: Seq[AttributeReference] = throw new IllegalStateException(
-    "RuntimeReplaceableAggregate.aggBufferAttributes should not be called")
-  def inputAggBufferAttributes: Seq[AttributeReference] = throw new IllegalStateException(
-    "RuntimeReplaceableAggregate.inputAggBufferAttributes should not be called")
+trait ReplaceableAggregateFunction extends AggregateFunction { self: RuntimeReplaceable =>

Review comment:
       how about `trait ReplaceableAggregateFunction extends RuntimeReplaceable { self: AggregateFunction `? Then it looks more like an add-on of `AggregateFunction`




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

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

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



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


[GitHub] [spark] HyukjinKwon commented on a change in pull request #35626: [SPARK-38240][SQL][FOLLOW-UP] Make RuntimeReplaceableAggregate as an add-on of RuntimeReplaceable

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/boolAggregates.scala
##########
@@ -56,7 +58,7 @@ case class BoolAnd(child: Expression) extends RuntimeReplaceableAggregate
   """,
   group = "agg_funcs",
   since = "3.0.0")
-case class BoolOr(child: Expression) extends RuntimeReplaceableAggregate
+case class BoolOr(child: Expression) extends ReplaceableAggregateFunction with RuntimeReplaceable

Review comment:
       I think this is fine because it fits in one line.




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

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

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



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


[GitHub] [spark] cloud-fan commented on a change in pull request #35626: [SPARK-38240][SQL][FOLLOW-UP] Make RuntimeReplaceableAggregate as an add-on of RuntimeReplaceable

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CountIf.scala
##########
@@ -34,8 +34,12 @@ import org.apache.spark.sql.types.{AbstractDataType, BooleanType}
   """,
   group = "agg_funcs",
   since = "3.0.0")
-case class CountIf(child: Expression) extends RuntimeReplaceableAggregate
-  with ImplicitCastInputTypes with UnaryLike[Expression] {
+case class CountIf(
+    child: Expression)
+  extends AggregateFunction
+  with ReplaceableAggregateFunction

Review comment:
       `extends AggregateFunction with ReplaceableAggregateFunction` looks a bit weird. Maybe the old name `RuntimeReplaceableAggregate` is better as a mix-in 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.

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

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



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


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #35626: [SPARK-38240][SQL][FOLLOW-UP] Make RuntimeReplaceableAggregate as an add-on of RuntimeReplaceable

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #35626:
URL: https://github.com/apache/spark/pull/35626#discussion_r813204144



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/boolAggregates.scala
##########
@@ -56,7 +58,7 @@ case class BoolAnd(child: Expression) extends RuntimeReplaceableAggregate
   """,
   group = "agg_funcs",
   since = "3.0.0")
-case class BoolOr(child: Expression) extends RuntimeReplaceableAggregate
+case class BoolOr(child: Expression) extends ReplaceableAggregateFunction with RuntimeReplaceable

Review comment:
       At this time, shall we split this line by moving `extends ...` to next line?




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

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

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



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


[GitHub] [spark] HyukjinKwon commented on pull request #35626: [SPARK-38240][SQL][FOLLOW-UP] Make RuntimeReplaceableAggregate as an add-on of AggregateFunction

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


   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.

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

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



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


[GitHub] [spark] HyukjinKwon commented on pull request #35626: [SPARK-38240][SQL][FOLLOW-UP] Make RuntimeReplaceableAggregate as an add-on of AggregateFunction

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


   cc @cloud-fan WDYT?


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

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

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



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


[GitHub] [spark] cloud-fan commented on a change in pull request #35626: [SPARK-38240][SQL][FOLLOW-UP] Make RuntimeReplaceableAggregate as an add-on of AggregateFunction

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/linearRegression.scala
##########
@@ -36,8 +36,13 @@ import org.apache.spark.sql.types.{AbstractDataType, NumericType}
   """,
   group = "agg_funcs",
   since = "3.3.0")
-case class RegrCount(left: Expression, right: Expression)
-  extends RuntimeReplaceableAggregate with ImplicitCastInputTypes with BinaryLike[Expression] {
+case class RegrCount(
+    left: Expression,
+    right: Expression)

Review comment:
       ```suggestion
   case class RegrCount(left: Expression, right: Expression)
   ```




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

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

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



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