[GitHub] [spark] cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] Implement a helper method for aliasing functions

2019-12-17 Thread GitBox
cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] 
Implement a helper method for aliasing functions
URL: https://github.com/apache/spark/pull/26808#discussion_r358854068
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
 ##
 @@ -321,4 +331,22 @@ object ApproximatePercentile {
   }
 
   val serializer: PercentileDigestSerializer = new PercentileDigestSerializer
+
+  def apply(
+child: Expression,
+percentageExpression: Expression,
+accuracyExpression: Expression,
+mutableAggBufferOffset: Int, inputAggBufferOffset: Int): 
ApproximatePercentile =
+new ApproximatePercentile("percentile_approx", child, percentageExpression,
+  accuracyExpression, mutableAggBufferOffset, inputAggBufferOffset)
+
+  def apply(
+child: 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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] Implement a helper method for aliasing functions

2019-12-17 Thread GitBox
cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] 
Implement a helper method for aliasing functions
URL: https://github.com/apache/spark/pull/26808#discussion_r358853945
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala
 ##
 @@ -35,9 +35,11 @@ import org.apache.spark.sql.types._
-3 days -11 hours -59 minutes -59 seconds
   """,
   since = "1.0.0")
-case class Average(child: Expression) extends DeclarativeAggregate with 
ImplicitCastInputTypes {
+case class Average(
+  funcName: String, child: 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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] Implement a helper method for aliasing functions

2019-12-17 Thread GitBox
cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] 
Implement a helper method for aliasing functions
URL: https://github.com/apache/spark/pull/26808#discussion_r358853809
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
 ##
 @@ -67,19 +67,25 @@ import org.apache.spark.sql.types._
   """,
   since = "2.1.0")
 case class ApproximatePercentile(
+funcName: String,
 child: Expression,
 percentageExpression: Expression,
 accuracyExpression: Expression,
 override val mutableAggBufferOffset: Int,
 override val inputAggBufferOffset: Int)
   extends TypedImperativeAggregate[PercentileDigest] with 
ImplicitCastInputTypes {
 
-  def this(child: Expression, percentageExpression: Expression, 
accuracyExpression: Expression) = {
-this(child, percentageExpression, accuracyExpression, 0, 0)
+  def this(
+funcName: String,
 
 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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] Implement a helper method for aliasing functions

2019-12-17 Thread GitBox
cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] 
Implement a helper method for aliasing functions
URL: https://github.com/apache/spark/pull/26808#discussion_r358853756
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
 ##
 @@ -321,4 +331,22 @@ object ApproximatePercentile {
   }
 
   val serializer: PercentileDigestSerializer = new PercentileDigestSerializer
+
+  def apply(
+child: Expression,
 
 Review comment:
   nit: 4 space 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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] Implement a helper method for aliasing functions

2019-12-16 Thread GitBox
cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] 
Implement a helper method for aliasing functions
URL: https://github.com/apache/spark/pull/26808#discussion_r358181344
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
 ##
 @@ -67,19 +67,25 @@ import org.apache.spark.sql.types._
   """,
   since = "2.1.0")
 case class ApproximatePercentile(
+funcName: String,
 child: Expression,
 percentageExpression: Expression,
 accuracyExpression: Expression,
 override val mutableAggBufferOffset: Int,
 override val inputAggBufferOffset: Int)
   extends TypedImperativeAggregate[PercentileDigest] with 
ImplicitCastInputTypes {
 
-  def this(child: Expression, percentageExpression: Expression, 
accuracyExpression: Expression) = {
-this(child, percentageExpression, accuracyExpression, 0, 0)
+  def this(
 
 Review comment:
   who calls this constructor?


This is an automated message from the 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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] Implement a helper method for aliasing functions

2019-12-16 Thread GitBox
cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] 
Implement a helper method for aliasing functions
URL: https://github.com/apache/spark/pull/26808#discussion_r358181384
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
 ##
 @@ -67,19 +67,25 @@ import org.apache.spark.sql.types._
   """,
   since = "2.1.0")
 case class ApproximatePercentile(
+funcName: String,
 child: Expression,
 percentageExpression: Expression,
 accuracyExpression: Expression,
 override val mutableAggBufferOffset: Int,
 override val inputAggBufferOffset: Int)
   extends TypedImperativeAggregate[PercentileDigest] with 
ImplicitCastInputTypes {
 
-  def this(child: Expression, percentageExpression: Expression, 
accuracyExpression: Expression) = {
-this(child, percentageExpression, accuracyExpression, 0, 0)
+  def this(
+funcName: String,
+child: Expression,
+percentageExpression: Expression,
+accuracyExpression: Expression) = {
+this(funcName, child, percentageExpression, accuracyExpression, 0, 0)
   }
 
-  def this(child: Expression, percentageExpression: Expression) = {
-this(child, percentageExpression, 
Literal(ApproximatePercentile.DEFAULT_PERCENTILE_ACCURACY))
+  def this(funcName: String, child: Expression, percentageExpression: 
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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] Implement a helper method for aliasing functions

2019-12-16 Thread GitBox
cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] 
Implement a helper method for aliasing functions
URL: https://github.com/apache/spark/pull/26808#discussion_r358181384
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
 ##
 @@ -67,19 +67,25 @@ import org.apache.spark.sql.types._
   """,
   since = "2.1.0")
 case class ApproximatePercentile(
+funcName: String,
 child: Expression,
 percentageExpression: Expression,
 accuracyExpression: Expression,
 override val mutableAggBufferOffset: Int,
 override val inputAggBufferOffset: Int)
   extends TypedImperativeAggregate[PercentileDigest] with 
ImplicitCastInputTypes {
 
-  def this(child: Expression, percentageExpression: Expression, 
accuracyExpression: Expression) = {
-this(child, percentageExpression, accuracyExpression, 0, 0)
+  def this(
+funcName: String,
+child: Expression,
+percentageExpression: Expression,
+accuracyExpression: Expression) = {
+this(funcName, child, percentageExpression, accuracyExpression, 0, 0)
   }
 
-  def this(child: Expression, percentageExpression: Expression) = {
-this(child, percentageExpression, 
Literal(ApproximatePercentile.DEFAULT_PERCENTILE_ACCURACY))
+  def this(funcName: String, child: Expression, percentageExpression: 
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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] Implement a helper method for aliasing functions

2019-12-16 Thread GitBox
cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] 
Implement a helper method for aliasing functions
URL: https://github.com/apache/spark/pull/26808#discussion_r358181344
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
 ##
 @@ -67,19 +67,25 @@ import org.apache.spark.sql.types._
   """,
   since = "2.1.0")
 case class ApproximatePercentile(
+funcName: String,
 child: Expression,
 percentageExpression: Expression,
 accuracyExpression: Expression,
 override val mutableAggBufferOffset: Int,
 override val inputAggBufferOffset: Int)
   extends TypedImperativeAggregate[PercentileDigest] with 
ImplicitCastInputTypes {
 
-  def this(child: Expression, percentageExpression: Expression, 
accuracyExpression: Expression) = {
-this(child, percentageExpression, accuracyExpression, 0, 0)
+  def this(
 
 Review comment:
   who calls this constructor?


This is an automated message from the 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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] Implement a helper method for aliasing functions

2019-12-12 Thread GitBox
cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] 
Implement a helper method for aliasing functions
URL: https://github.com/apache/spark/pull/26808#discussion_r357023338
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
 ##
 @@ -635,7 +635,6 @@ object FunctionRegistry {
   (implicit tag: ClassTag[T]): (String, (ExpressionInfo, FunctionBuilder)) 
= {
 val constructors = tag.runtimeClass.getConstructors
   .filter(_.getParameterTypes.head == classOf[String])
-assert(constructors.length == 1)
 
 Review comment:
   sounds good


This is an automated message from the 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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] Implement a helper method for aliasing functions

2019-12-11 Thread GitBox
cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] 
Implement a helper method for aliasing functions
URL: https://github.com/apache/spark/pull/26808#discussion_r356561965
 
 

 ##
 File path: 
sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out
 ##
 @@ -101,7 +101,7 @@ struct
 -- !query 12
 SELECT udf(var_samp(b)) FROM aggtest
 -- !query 12 schema
-struct
+struct
 
 Review comment:
   This looks weird. Looking at `Expression.toString`, I think we should 
override `flatArguments` in these expressions with alias to exclude the 
funcName parameter.


This is an automated message from the 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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] Implement a helper method for aliasing functions

2019-12-10 Thread GitBox
cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] 
Implement a helper method for aliasing functions
URL: https://github.com/apache/spark/pull/26808#discussion_r356432760
 
 

 ##
 File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala
 ##
 @@ -254,7 +254,8 @@ object StatFunctions extends Logging {
 stats.toLowerCase(Locale.ROOT) match {
   case "count" => (child: Expression) => 
Count(child).toAggregateExpression()
   case "mean" => (child: Expression) => 
Average(child).toAggregateExpression()
-  case "stddev" => (child: Expression) => 
StddevSamp(child).toAggregateExpression()
+  case "stddev" => (child: Expression) =>
+StddevSamp("stddev_samp", child).toAggregateExpression()
 
 Review comment:
   shall we use `stddev` 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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] Implement a helper method for aliasing functions

2019-12-10 Thread GitBox
cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] 
Implement a helper method for aliasing functions
URL: https://github.com/apache/spark/pull/26808#discussion_r356432466
 
 

 ##
 File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala
 ##
 @@ -254,7 +254,8 @@ object StatFunctions extends Logging {
 stats.toLowerCase(Locale.ROOT) match {
   case "count" => (child: Expression) => 
Count(child).toAggregateExpression()
   case "mean" => (child: Expression) => 
Average(child).toAggregateExpression()
-  case "stddev" => (child: Expression) => 
StddevSamp(child).toAggregateExpression()
+  case "stddev" => (child: Expression) =>
+StddevSamp("stddev_samp", child).toAggregateExpression()
 
 Review comment:
   can we create a `object StddevSamp` and define the default name in the 
`apply` method?


This is an automated message from the 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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] Implement a helper method for aliasing functions

2019-12-10 Thread GitBox
cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] 
Implement a helper method for aliasing functions
URL: https://github.com/apache/spark/pull/26808#discussion_r356432466
 
 

 ##
 File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala
 ##
 @@ -254,7 +254,8 @@ object StatFunctions extends Logging {
 stats.toLowerCase(Locale.ROOT) match {
   case "count" => (child: Expression) => 
Count(child).toAggregateExpression()
   case "mean" => (child: Expression) => 
Average(child).toAggregateExpression()
-  case "stddev" => (child: Expression) => 
StddevSamp(child).toAggregateExpression()
+  case "stddev" => (child: Expression) =>
+StddevSamp("stddev_samp", child).toAggregateExpression()
 
 Review comment:
   can we create a `object StddevSamp` and define the default name in the 
`apply` method?


This is an automated message from the 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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] Implement a helper method for aliasing functions

2019-12-10 Thread GitBox
cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] 
Implement a helper method for aliasing functions
URL: https://github.com/apache/spark/pull/26808#discussion_r356432239
 
 

 ##
 File path: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala
 ##
 @@ -794,12 +794,18 @@ class ExpressionParserSuite extends AnalysisTest {
   }
 
   test("Support respect nulls keywords for first_value and last_value") {
-assertEqual("first_value(a ignore nulls)", First('a, 
Literal(true)).toAggregateExpression())
-assertEqual("first_value(a respect nulls)", First('a, 
Literal(false)).toAggregateExpression())
-assertEqual("first_value(a)", First('a, 
Literal(false)).toAggregateExpression())
-assertEqual("last_value(a ignore nulls)", Last('a, 
Literal(true)).toAggregateExpression())
-assertEqual("last_value(a respect nulls)", Last('a, 
Literal(false)).toAggregateExpression())
-assertEqual("last_value(a)", Last('a, 
Literal(false)).toAggregateExpression())
+assertEqual("first_value(a ignore nulls)",
+  First('a, Literal(true)).toAggregateExpression())
 
 Review comment:
   ditto, unnecessary change


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] Implement a helper method for aliasing functions

2019-12-10 Thread GitBox
cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] 
Implement a helper method for aliasing functions
URL: https://github.com/apache/spark/pull/26808#discussion_r356432177
 
 

 ##
 File path: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/LastTestSuite.scala
 ##
 @@ -24,7 +24,8 @@ import org.apache.spark.sql.types.IntegerType
 class LastTestSuite extends SparkFunSuite {
   val input = AttributeReference("input", IntegerType, nullable = true)()
   val evaluator = DeclarativeAggregateEvaluator(Last(input, Literal(false)), 
Seq(input))
-  val evaluatorIgnoreNulls = DeclarativeAggregateEvaluator(Last(input, 
Literal(true)), Seq(input))
+  val evaluatorIgnoreNulls = DeclarativeAggregateEvaluator(
+Last(input, Literal(true)), Seq(input))
 
 Review comment:
   nit: unnecessary change


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] Implement a helper method for aliasing functions

2019-12-10 Thread GitBox
cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] 
Implement a helper method for aliasing functions
URL: https://github.com/apache/spark/pull/26808#discussion_r356135812
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##
 @@ -198,7 +198,7 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] 
{
 
 // Select the result of the first aggregate in the last aggregate.
 val result = AggregateExpression(
-  aggregate.First(evalWithinGroup(regularGroupId, 
operator.toAttribute), Literal(true)),
+  new aggregate.First(evalWithinGroup(regularGroupId, 
operator.toAttribute), Literal(true)),
 
 Review comment:
   so adding more constructors doesn't help us to reduce diff, how about
   ```
   case class First(funcName: String, child: Expression, ignoreNullsExpr: 
Expression) {
 def this(funcName: String, child: Expression)
   }
   object First {
 def apply(child: Expression, ignoreNullsExpr: Expression) ...
 def apply(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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] Implement a helper method for aliasing functions

2019-12-09 Thread GitBox
cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] 
Implement a helper method for aliasing functions
URL: https://github.com/apache/spark/pull/26808#discussion_r355816787
 
 

 ##
 File path: sql/core/src/main/scala/org/apache/spark/sql/functions.scala
 ##
 @@ -431,7 +431,7 @@ object functions {
* @since 2.0.0
*/
   def first(e: Column, ignoreNulls: Boolean): Column = withAggregateFunction {
-new First(e.expr, Literal(ignoreNulls))
+new First("first", e.expr, Literal(ignoreNulls))
 
 Review comment:
   We can add a constructor to provide a default name.


This is an automated message from the 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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] Implement a helper method for aliasing functions

2019-12-09 Thread GitBox
cloud-fan commented on a change in pull request #26808: [SPARK-30184][SQL] 
Implement a helper method for aliasing functions
URL: https://github.com/apache/spark/pull/26808#discussion_r355816405
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
 ##
 @@ -302,14 +302,14 @@ object FunctionRegistry {
 expression[Skewness]("skewness"),
 expression[ApproximatePercentile]("percentile_approx"),
 expression[ApproximatePercentile]("approx_percentile"),
-expression[StddevSamp]("std"),
-expression[StddevSamp]("stddev"),
+expressionWithAlias[StddevSamp]("std"),
+expressionWithAlias[StddevSamp]("stddev"),
 expression[StddevPop]("stddev_pop"),
-expression[StddevSamp]("stddev_samp"),
+expressionWithAlias[StddevSamp]("stddev_samp"),
 
 Review comment:
   can we put them closer if they use the same 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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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