diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionDescription.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionDescription.java index 9e10f27d59d55c8bef39b6f514e8c13ea15394e1..62a2ce47d0ce60d75df093f325e9f34a8592c9f6 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionDescription.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionDescription.java @@ -39,5 +39,5 @@ import java.lang.annotation.RetentionPolicy; @Retention(RetentionPolicy.RUNTIME) public @interface ExpressionDescription { String usage() default "_FUNC_ is undocumented"; - String extended() default "No example for _FUNC_."; + String extended() default "\n No example/argument for _FUNC_.\n"; } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflection.scala index fe24c0489fc98371eb4598fc6801bae37c734942..40f1b148f92878a15c869e990b93449bf7fb16d0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflection.scala @@ -43,11 +43,15 @@ import org.apache.spark.util.Utils * and the second element should be a literal string for the method name, * and the remaining are input arguments to the Java method. */ -// scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(class,method[,arg1[,arg2..]]) calls method with reflection", - extended = "> SELECT _FUNC_('java.util.UUID', 'randomUUID');\n c33fb387-8500-4bfa-81d2-6e0e3e930df2") -// scalastyle:on line.size.limit + usage = "_FUNC_(class, method[, arg1[, arg2 ..]]) - Calls a method with reflection.", + extended = """ + Examples: + > SELECT _FUNC_('java.util.UUID', 'randomUUID'); + c33fb387-8500-4bfa-81d2-6e0e3e930df2 + > SELECT _FUNC_('java.util.UUID', 'fromString', 'a5cf6c42-0c85-418f-af6c-3e4e5b1328f2'); + a5cf6c42-0c85-418f-af6c-3e4e5b1328f2 + """) case class CallMethodViaReflection(children: Seq[Expression]) extends Expression with CodegenFallback { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 58fd65f62ffe796b24b14e1265a117f6c28f4f56..4db1ae6faa1599465c570d7c97de6abafc45d6fe 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -114,8 +114,12 @@ object Cast { /** Cast the child expression to the target data type. */ @ExpressionDescription( - usage = " - Cast value v to the target data type.", - extended = "> SELECT _FUNC_('10' as int);\n 10") + usage = "_FUNC_(expr AS type) - Casts the value `expr` to the target data type `type`.", + extended = """ + Examples: + > SELECT _FUNC_('10' as int); + 10 + """) case class Cast(child: Expression, dataType: DataType) extends UnaryExpression with NullIntolerant { override def toString: String = s"cast($child as ${dataType.simpleString})" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InputFileName.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InputFileName.scala index b6c12c535111922ece06c23dcf6ef698fd0ac7ba..b7fb285133bfcde67f8d84ea81b2709c7bf1aee4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InputFileName.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InputFileName.scala @@ -27,8 +27,7 @@ import org.apache.spark.unsafe.types.UTF8String * Expression that returns the name of the current file being read. */ @ExpressionDescription( - usage = "_FUNC_() - Returns the name of the current file being read if available", - extended = "> SELECT _FUNC_();\n ''") + usage = "_FUNC_() - Returns the name of the current file being read if available.") case class InputFileName() extends LeafExpression with Nondeterministic { override def nullable: Boolean = true diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala index 72b8dcca26e2f64c0cf3f93288818d786ee07f82..32358a99e7ce757ce4c87ca65b3b744749320a81 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala @@ -33,13 +33,13 @@ import org.apache.spark.sql.types.{DataType, LongType} * Since this expression is stateful, it cannot be a case object. */ @ExpressionDescription( - usage = - """_FUNC_() - Returns monotonically increasing 64-bit integers. - The generated ID is guaranteed to be monotonically increasing and unique, but not consecutive. - The current implementation puts the partition ID in the upper 31 bits, and the lower 33 bits - represent the record number within each partition. The assumption is that the data frame has - less than 1 billion partitions, and each partition has less than 8 billion records.""", - extended = "> SELECT _FUNC_();\n 0") + usage = """ + _FUNC_() - Returns monotonically increasing 64-bit integers. The generated ID is guaranteed + to be monotonically increasing and unique, but not consecutive. The current implementation + puts the partition ID in the upper 31 bits, and the lower 33 bits represent the record number + within each partition. The assumption is that the data frame has less than 1 billion + partitions, and each partition has less than 8 billion records. + """) case class MonotonicallyIncreasingID() extends LeafExpression with Nondeterministic { /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SparkPartitionID.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SparkPartitionID.scala index 6bef473cac060be29e20b41404bc1a3760778feb..8db7efdbb5dd4db9c53a3aba2a5781e21f6a3bb5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SparkPartitionID.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SparkPartitionID.scala @@ -25,8 +25,7 @@ import org.apache.spark.sql.types.{DataType, IntegerType} * Expression that returns the current partition id. */ @ExpressionDescription( - usage = "_FUNC_() - Returns the current partition id", - extended = "> SELECT _FUNC_();\n 0") + usage = "_FUNC_() - Returns the current partition id.") case class SparkPartitionID() extends LeafExpression with Nondeterministic { override def nullable: Boolean = false diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala index f91ff87fc1c0164dcf979655abe40198260ae90a..692cbd7c0d32c65a8e46d44aad32fedf563a8081 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala @@ -49,21 +49,23 @@ import org.apache.spark.sql.types._ * DEFAULT_PERCENTILE_ACCURACY. */ @ExpressionDescription( - usage = - """ - _FUNC_(col, percentage [, accuracy]) - Returns the approximate percentile value of numeric + usage = """ + _FUNC_(col, percentage [, accuracy]) - Returns the approximate percentile value of numeric column `col` at the given percentage. The value of percentage must be between 0.0 - and 1.0. The `accuracy` parameter (default: 10000) is a positive integer literal which + and 1.0. The `accuracy` parameter (default: 10000) is a positive numeric literal which controls approximation accuracy at the cost of memory. Higher value of `accuracy` yields better accuracy, `1.0/accuracy` is the relative error of the approximation. - - _FUNC_(col, array(percentage1 [, percentage2]...) [, accuracy]) - Returns the approximate - percentile array of column `col` at the given percentage array. Each value of the - percentage array must be between 0.0 and 1.0. The `accuracy` parameter (default: 10000) is - a positive integer literal which controls approximation accuracy at the cost of memory. - Higher value of `accuracy` yields better accuracy, `1.0/accuracy` is the relative error of - the approximation. - """) + When `percentage` is an array, each value of the percentage array must be between 0.0 and 1.0. + In this case, returns the approximate percentile array of column `col` at the given + percentage array. + """, + extended = """ + Examples: + > SELECT percentile_approx(10.0, array(0.5, 0.4, 0.1), 100); + [10.0,10.0,10.0] + > SELECT percentile_approx(10.0, 0.5, 100); + 10.0 + """) case class ApproximatePercentile( child: Expression, percentageExpression: Expression, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala index ff70774847830c36d1707d7c062a309154d3e2f5..d523420530c2caf9b871ed9c6739df27d9576200 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.util.TypeUtils import org.apache.spark.sql.types._ @ExpressionDescription( - usage = "_FUNC_(x) - Returns the mean calculated from values of a group.") + usage = "_FUNC_(expr) - Returns the mean calculated from values of a group.") case class Average(child: Expression) extends DeclarativeAggregate { override def prettyName: String = "avg" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala index 17a7c6dce89cab5c469fc54687ac68203ddb8d3f..302054708ccb5cd694f7a3ad2979193ff1d6788f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala @@ -132,7 +132,7 @@ abstract class CentralMomentAgg(child: Expression) extends DeclarativeAggregate // Compute the population standard deviation of a column // scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(x) - Returns the population standard deviation calculated from values of a group.") + usage = "_FUNC_(expr) - Returns the population standard deviation calculated from values of a group.") // scalastyle:on line.size.limit case class StddevPop(child: Expression) extends CentralMomentAgg(child) { @@ -147,8 +147,10 @@ case class StddevPop(child: Expression) extends CentralMomentAgg(child) { } // Compute the sample standard deviation of a column +// scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(x) - Returns the sample standard deviation calculated from values of a group.") + usage = "_FUNC_(expr) - Returns the sample standard deviation calculated from values of a group.") +// scalastyle:on line.size.limit case class StddevSamp(child: Expression) extends CentralMomentAgg(child) { override protected def momentOrder = 2 @@ -164,7 +166,7 @@ case class StddevSamp(child: Expression) extends CentralMomentAgg(child) { // Compute the population variance of a column @ExpressionDescription( - usage = "_FUNC_(x) - Returns the population variance calculated from values of a group.") + usage = "_FUNC_(expr) - Returns the population variance calculated from values of a group.") case class VariancePop(child: Expression) extends CentralMomentAgg(child) { override protected def momentOrder = 2 @@ -179,7 +181,7 @@ case class VariancePop(child: Expression) extends CentralMomentAgg(child) { // Compute the sample variance of a column @ExpressionDescription( - usage = "_FUNC_(x) - Returns the sample variance calculated from values of a group.") + usage = "_FUNC_(expr) - Returns the sample variance calculated from values of a group.") case class VarianceSamp(child: Expression) extends CentralMomentAgg(child) { override protected def momentOrder = 2 @@ -194,7 +196,7 @@ case class VarianceSamp(child: Expression) extends CentralMomentAgg(child) { } @ExpressionDescription( - usage = "_FUNC_(x) - Returns the Skewness value calculated from values of a group.") + usage = "_FUNC_(expr) - Returns the skewness value calculated from values of a group.") case class Skewness(child: Expression) extends CentralMomentAgg(child) { override def prettyName: String = "skewness" @@ -209,7 +211,7 @@ case class Skewness(child: Expression) extends CentralMomentAgg(child) { } @ExpressionDescription( - usage = "_FUNC_(x) - Returns the Kurtosis value calculated from values of a group.") + usage = "_FUNC_(expr) - Returns the kurtosis value calculated from values of a group.") case class Kurtosis(child: Expression) extends CentralMomentAgg(child) { override protected def momentOrder = 4 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Corr.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Corr.scala index e29265e2f41e10ed2b1a69fb8d695e98eb7eda54..657f519d2a05efe3058fe519181592ddbf12ad41 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Corr.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Corr.scala @@ -28,8 +28,10 @@ import org.apache.spark.sql.types._ * Definition of Pearson correlation can be found at * http://en.wikipedia.org/wiki/Pearson_product-moment_correlation_coefficient */ +// scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(x,y) - Returns Pearson coefficient of correlation between a set of number pairs.") + usage = "_FUNC_(expr1, expr2) - Returns Pearson coefficient of correlation between a set of number pairs.") +// scalastyle:on line.size.limit case class Corr(x: Expression, y: Expression) extends DeclarativeAggregate { override def children: Seq[Expression] = Seq(x, y) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Count.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Count.scala index 17ae012af79bea54ee9f4f04dc1786114509fe17..bcae0dc0754c4fed33acf42dd88b53a0c4785a7a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Count.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Count.scala @@ -23,9 +23,13 @@ import org.apache.spark.sql.types._ // scalastyle:off line.size.limit @ExpressionDescription( - usage = """_FUNC_(*) - Returns the total number of retrieved rows, including rows containing NULL values. - _FUNC_(expr) - Returns the number of rows for which the supplied expression is non-NULL. - _FUNC_(DISTINCT expr[, expr...]) - Returns the number of rows for which the supplied expression(s) are unique and non-NULL.""") + usage = """ + _FUNC_(*) - Returns the total number of retrieved rows, including rows containing null. + + _FUNC_(expr) - Returns the number of rows for which the supplied expression is non-null. + + _FUNC_(DISTINCT expr[, expr...]) - Returns the number of rows for which the supplied expression(s) are unique and non-null. + """) // scalastyle:on line.size.limit case class Count(children: Seq[Expression]) extends DeclarativeAggregate { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala index d80afbebf7404842295d3db163e2f9cf00d05ce7..ae5ed779700b6d1a1eef0da114fb9dc77936a6ae 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala @@ -77,7 +77,7 @@ abstract class Covariance(x: Expression, y: Expression) extends DeclarativeAggre } @ExpressionDescription( - usage = "_FUNC_(x,y) - Returns the population covariance of a set of number pairs.") + usage = "_FUNC_(expr1, expr2) - Returns the population covariance of a set of number pairs.") case class CovPopulation(left: Expression, right: Expression) extends Covariance(left, right) { override val evaluateExpression: Expression = { If(n === Literal(0.0), Literal.create(null, DoubleType), @@ -88,7 +88,7 @@ case class CovPopulation(left: Expression, right: Expression) extends Covariance @ExpressionDescription( - usage = "_FUNC_(x,y) - Returns the sample covariance of a set of number pairs.") + usage = "_FUNC_(expr1, expr2) - Returns the sample covariance of a set of number pairs.") case class CovSample(left: Expression, right: Expression) extends Covariance(left, right) { override val evaluateExpression: Expression = { If(n === Literal(0.0), Literal.create(null, DoubleType), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala index d702c08cfd342fc0e3ffc79a05e5a47155e26161..29b894798000440d7380c369b66887076bd66ae6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala @@ -29,10 +29,10 @@ import org.apache.spark.sql.types._ * a single partition, and we use a single reducer to do the aggregation.). */ @ExpressionDescription( - usage = """_FUNC_(expr) - Returns the first value of `child` for a group of rows. - _FUNC_(expr,isIgnoreNull=false) - Returns the first value of `child` for a group of rows. - If isIgnoreNull is true, returns only non-null values. - """) + usage = """ + _FUNC_(expr[, isIgnoreNull]) - Returns the first value of `expr` for a group of rows. + If `isIgnoreNull` is true, returns only non-null values. + """) case class First(child: Expression, ignoreNullsExpr: Expression) extends DeclarativeAggregate { def this(child: Expression) = this(child, Literal.create(false, BooleanType)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlus.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlus.scala index 83c8d400c5d6a20d469126add694e640e2a57e6b..b9862aa04fcd9d98de6fc487bf31e012db51ca68 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlus.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlus.scala @@ -47,10 +47,10 @@ import org.apache.spark.sql.types._ */ // scalastyle:on @ExpressionDescription( - usage = """_FUNC_(expr) - Returns the estimated cardinality by HyperLogLog++. - _FUNC_(expr, relativeSD=0.05) - Returns the estimated cardinality by HyperLogLog++ - with relativeSD, the maximum estimation error allowed. - """) + usage = """ + _FUNC_(expr[, relativeSD]) - Returns the estimated cardinality by HyperLogLog++. + `relativeSD` defines the maximum estimation error allowed. + """) case class HyperLogLogPlusPlus( child: Expression, relativeSD: Double = 0.05, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala index 8579f7292d3abf843bac40d251d6be53ba81af8d..b0a363e7d6dce03482105afa8ea3f6350c88705e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala @@ -29,7 +29,10 @@ import org.apache.spark.sql.types._ * a single partition, and we use a single reducer to do the aggregation.). */ @ExpressionDescription( - usage = "_FUNC_(expr,isIgnoreNull) - Returns the last value of `child` for a group of rows.") + usage = """ + _FUNC_(expr[, isIgnoreNull]) - Returns the last value of `expr` for a group of rows. + If `isIgnoreNull` is true, returns only non-null values. + """) case class Last(child: Expression, ignoreNullsExpr: Expression) extends DeclarativeAggregate { def this(child: Expression) = this(child, Literal.create(false, BooleanType)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Max.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Max.scala index c534fe495fc13cacc850c971cfaf3630b0360ae4..f32c9c677a864108f0e5071f200878f44e579a26 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Max.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Max.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.util.TypeUtils import org.apache.spark.sql.types._ @ExpressionDescription( - usage = "_FUNC_(expr) - Returns the maximum value of expr.") + usage = "_FUNC_(expr) - Returns the maximum value of `expr`.") case class Max(child: Expression) extends DeclarativeAggregate { override def children: Seq[Expression] = child :: Nil diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Min.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Min.scala index 35289b468183c6285f99aed4ce04b1ab3546d019..9ef42b96975afcdba7caf65982d8237e587d93bf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Min.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Min.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.util.TypeUtils import org.apache.spark.sql.types._ @ExpressionDescription( - usage = "_FUNC_(expr) - Returns the minimum value of expr.") + usage = "_FUNC_(expr) - Returns the minimum value of `expr`.") case class Min(child: Expression) extends DeclarativeAggregate { override def children: Seq[Expression] = child :: Nil diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala index ad217f25b5a26635f8ed9576790e6de4516f2f6a..f3731d40058e33bd2af337f78cc15547c57b320e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.util.TypeUtils import org.apache.spark.sql.types._ @ExpressionDescription( - usage = "_FUNC_(x) - Returns the sum calculated from values of a group.") + usage = "_FUNC_(expr) - Returns the sum calculated from values of a group.") case class Sum(child: Expression) extends DeclarativeAggregate { override def children: Seq[Expression] = child :: Nil diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala index 89eb864e947026f5a4ff00bfeab6b167988d5b4e..d2880d58aefe1853a2820ae4d2b7cd6ba8189304 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala @@ -106,7 +106,7 @@ case class CollectList( } /** - * Collect a list of unique elements. + * Collect a set of unique elements. */ @ExpressionDescription( usage = "_FUNC_(expr) - Collects and returns a set of unique elements.") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index 6f3db79622fa2aab5f961dddb7c71e662991c5b8..4870093e9250f79fcf883703859b062f32aa71ba 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -25,7 +25,12 @@ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval @ExpressionDescription( - usage = "_FUNC_(a) - Returns -a.") + usage = "_FUNC_(expr) - Returns the negated value of `expr`.", + extended = """ + Examples: + > SELECT _FUNC_(1); + -1 + """) case class UnaryMinus(child: Expression) extends UnaryExpression with ExpectsInputTypes with NullIntolerant { @@ -62,7 +67,7 @@ case class UnaryMinus(child: Expression) extends UnaryExpression } @ExpressionDescription( - usage = "_FUNC_(a) - Returns a.") + usage = "_FUNC_(expr) - Returns the value of `expr`.") case class UnaryPositive(child: Expression) extends UnaryExpression with ExpectsInputTypes with NullIntolerant { override def prettyName: String = "positive" @@ -84,7 +89,11 @@ case class UnaryPositive(child: Expression) */ @ExpressionDescription( usage = "_FUNC_(expr) - Returns the absolute value of the numeric value.", - extended = "> SELECT _FUNC_('-1');\n 1") + extended = """ + Examples: + > SELECT _FUNC_(-1); + 1 + """) case class Abs(child: Expression) extends UnaryExpression with ExpectsInputTypes with NullIntolerant { @@ -131,7 +140,12 @@ object BinaryArithmetic { } @ExpressionDescription( - usage = "a _FUNC_ b - Returns a+b.") + usage = "expr1 _FUNC_ expr2 - Returns `expr1`+`expr2`.", + extended = """ + Examples: + > SELECT 1 _FUNC_ 2; + 3 + """) case class Add(left: Expression, right: Expression) extends BinaryArithmetic with NullIntolerant { override def inputType: AbstractDataType = TypeCollection.NumericAndInterval @@ -162,7 +176,12 @@ case class Add(left: Expression, right: Expression) extends BinaryArithmetic wit } @ExpressionDescription( - usage = "a _FUNC_ b - Returns a-b.") + usage = "expr1 _FUNC_ expr2 - Returns `expr1`-`expr2`.", + extended = """ + Examples: + > SELECT 2 _FUNC_ 1; + 1 + """) case class Subtract(left: Expression, right: Expression) extends BinaryArithmetic with NullIntolerant { @@ -194,7 +213,12 @@ case class Subtract(left: Expression, right: Expression) } @ExpressionDescription( - usage = "a _FUNC_ b - Multiplies a by b.") + usage = "expr1 _FUNC_ expr2 - Returns `expr1`*`expr2`.", + extended = """ + Examples: + > SELECT 2 _FUNC_ 3; + 6 + """) case class Multiply(left: Expression, right: Expression) extends BinaryArithmetic with NullIntolerant { @@ -208,9 +232,17 @@ case class Multiply(left: Expression, right: Expression) protected override def nullSafeEval(input1: Any, input2: Any): Any = numeric.times(input1, input2) } +// scalastyle:off line.size.limit @ExpressionDescription( - usage = "a _FUNC_ b - Divides a by b.", - extended = "> SELECT 3 _FUNC_ 2;\n 1.5") + usage = "expr1 _FUNC_ expr2 - Returns `expr1`/`expr2`. It always performs floating point division.", + extended = """ + Examples: + > SELECT 3 _FUNC_ 2; + 1.5 + > SELECT 2L _FUNC_ 2L; + 1.0 + """) +// scalastyle:on line.size.limit case class Divide(left: Expression, right: Expression) extends BinaryArithmetic with NullIntolerant { @@ -286,7 +318,12 @@ case class Divide(left: Expression, right: Expression) } @ExpressionDescription( - usage = "a _FUNC_ b - Returns the remainder when dividing a by b.") + usage = "expr1 _FUNC_ expr2 - Returns the remainder after `expr1`/`expr2`.", + extended = """ + Examples: + > SELECT 2 _FUNC_ 1.8; + 0.2 + """) case class Remainder(left: Expression, right: Expression) extends BinaryArithmetic with NullIntolerant { @@ -367,8 +404,14 @@ case class Remainder(left: Expression, right: Expression) } @ExpressionDescription( - usage = "_FUNC_(a, b) - Returns the positive modulo", - extended = "> SELECT _FUNC_(10,3);\n 1") + usage = "_FUNC_(expr1, expr2) - Returns the positive value of `expr1` mod `expr2`.", + extended = """ + Examples: + > SELECT _FUNC_(10, 3); + 1 + > SELECT _FUNC_(-10, 3); + 2 + """) case class Pmod(left: Expression, right: Expression) extends BinaryArithmetic with NullIntolerant { override def toString: String = s"pmod($left, $right)" @@ -471,7 +514,12 @@ case class Pmod(left: Expression, right: Expression) extends BinaryArithmetic wi * It takes at least 2 parameters, and returns null iff all parameters are null. */ @ExpressionDescription( - usage = "_FUNC_(n1, ...) - Returns the least value of all parameters, skipping null values.") + usage = "_FUNC_(expr, ...) - Returns the least value of all parameters, skipping null values.", + extended = """ + Examples: + > SELECT _FUNC_(10, 9, 2, 4, 3); + 2 + """) case class Least(children: Seq[Expression]) extends Expression { override def nullable: Boolean = children.forall(_.nullable) @@ -531,7 +579,12 @@ case class Least(children: Seq[Expression]) extends Expression { * It takes at least 2 parameters, and returns null iff all parameters are null. */ @ExpressionDescription( - usage = "_FUNC_(n1, ...) - Returns the greatest value of all parameters, skipping null values.") + usage = "_FUNC_(expr, ...) - Returns the greatest value of all parameters, skipping null values.", + extended = """ + Examples: + > SELECT _FUNC_(10, 9, 2, 4, 3); + 10 + """) case class Greatest(children: Seq[Expression]) extends Expression { override def nullable: Boolean = children.forall(_.nullable) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwiseExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwiseExpressions.scala index 3a0a882e3876e64f746536e16daddf5f23116b57..291804077143352b9f7ffbe8f72970bee399f4a5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwiseExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwiseExpressions.scala @@ -27,8 +27,12 @@ import org.apache.spark.sql.types._ * Code generation inherited from BinaryArithmetic. */ @ExpressionDescription( - usage = "a _FUNC_ b - Bitwise AND.", - extended = "> SELECT 3 _FUNC_ 5; 1") + usage = "expr1 _FUNC_ expr2 - Returns the result of bitwise AND of `expr1` and `expr2`.", + extended = """ + Examples: + > SELECT 3 _FUNC_ 5; + 1 + """) case class BitwiseAnd(left: Expression, right: Expression) extends BinaryArithmetic { override def inputType: AbstractDataType = IntegralType @@ -55,8 +59,12 @@ case class BitwiseAnd(left: Expression, right: Expression) extends BinaryArithme * Code generation inherited from BinaryArithmetic. */ @ExpressionDescription( - usage = "a _FUNC_ b - Bitwise OR.", - extended = "> SELECT 3 _FUNC_ 5; 7") + usage = "expr1 _FUNC_ expr2 - Returns the result of bitwise OR of `expr1` and `expr2`.", + extended = """ + Examples: + > SELECT 3 _FUNC_ 5; + 7 + """) case class BitwiseOr(left: Expression, right: Expression) extends BinaryArithmetic { override def inputType: AbstractDataType = IntegralType @@ -83,8 +91,12 @@ case class BitwiseOr(left: Expression, right: Expression) extends BinaryArithmet * Code generation inherited from BinaryArithmetic. */ @ExpressionDescription( - usage = "a _FUNC_ b - Bitwise exclusive OR.", - extended = "> SELECT 3 _FUNC_ 5; 2") + usage = "expr1 _FUNC_ expr2 - Returns the result of bitwise exclusive OR of `expr1` and `expr2`.", + extended = """ + Examples: + > SELECT 3 _FUNC_ 5; + 2 + """) case class BitwiseXor(left: Expression, right: Expression) extends BinaryArithmetic { override def inputType: AbstractDataType = IntegralType @@ -109,8 +121,12 @@ case class BitwiseXor(left: Expression, right: Expression) extends BinaryArithme * A function that calculates bitwise not(~) of a number. */ @ExpressionDescription( - usage = "_FUNC_ b - Bitwise NOT.", - extended = "> SELECT _FUNC_ 0; -1") + usage = "_FUNC_ expr - Returns the result of bitwise NOT of `expr`.", + extended = """ + Examples: + > SELECT _FUNC_ 0; + -1 + """) case class BitwiseNot(child: Expression) extends UnaryExpression with ExpectsInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(IntegralType) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index f56bb39d10791fbde0606c62cce306913565a938..c863ba434120d6419fee943c23e7dc26b295493d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -28,8 +28,12 @@ import org.apache.spark.sql.types._ * Given an array or map, returns its size. Returns -1 if null. */ @ExpressionDescription( - usage = "_FUNC_(expr) - Returns the size of an array or a map.", - extended = " > SELECT _FUNC_(array('b', 'd', 'c', 'a'));\n 4") + usage = "_FUNC_(expr) - Returns the size of an array or a map. Returns -1 if null.", + extended = """ + Examples: + > SELECT _FUNC_(array('b', 'd', 'c', 'a')); + 4 + """) case class Size(child: Expression) extends UnaryExpression with ExpectsInputTypes { override def dataType: DataType = IntegerType override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(ArrayType, MapType)) @@ -60,7 +64,11 @@ case class Size(child: Expression) extends UnaryExpression with ExpectsInputType */ @ExpressionDescription( usage = "_FUNC_(map) - Returns an unordered array containing the keys of the map.", - extended = " > SELECT _FUNC_(map(1, 'a', 2, 'b'));\n [1,2]") + extended = """ + Examples: + > SELECT _FUNC_(map(1, 'a', 2, 'b')); + [1,2] + """) case class MapKeys(child: Expression) extends UnaryExpression with ExpectsInputTypes { @@ -84,7 +92,11 @@ case class MapKeys(child: Expression) */ @ExpressionDescription( usage = "_FUNC_(map) - Returns an unordered array containing the values of the map.", - extended = " > SELECT _FUNC_(map(1, 'a', 2, 'b'));\n [\"a\",\"b\"]") + extended = """ + Examples: + > SELECT _FUNC_(map(1, 'a', 2, 'b')); + ["a","b"] + """) case class MapValues(child: Expression) extends UnaryExpression with ExpectsInputTypes { @@ -109,8 +121,12 @@ case class MapValues(child: Expression) */ // scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(array(obj1, obj2, ...), ascendingOrder) - Sorts the input array in ascending order according to the natural ordering of the array elements.", - extended = " > SELECT _FUNC_(array('b', 'd', 'c', 'a'), true);\n 'a', 'b', 'c', 'd'") + usage = "_FUNC_(array[, ascendingOrder]) - Sorts the input array in ascending or descending order according to the natural ordering of the array elements.", + extended = """ + Examples: + > SELECT _FUNC_(array('b', 'd', 'c', 'a'), true); + ["a","b","c","d"] + """) // scalastyle:on line.size.limit case class SortArray(base: Expression, ascendingOrder: Expression) extends BinaryExpression with ExpectsInputTypes with CodegenFallback { @@ -200,8 +216,12 @@ case class SortArray(base: Expression, ascendingOrder: Expression) * Checks if the array (left) has the element (right) */ @ExpressionDescription( - usage = "_FUNC_(array, value) - Returns TRUE if the array contains the value.", - extended = " > SELECT _FUNC_(array(1, 2, 3), 2);\n true") + usage = "_FUNC_(array, value) - Returns true if the array contains the value.", + extended = """ + Examples: + > SELECT _FUNC_(array(1, 2, 3), 2); + true + """) case class ArrayContains(left: Expression, right: Expression) extends BinaryExpression with ImplicitCastInputTypes { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala index dbfb2996ec9d595cc9a88d65cd7d44e0b4d4d5fe..c9f36649ec8ee5266eb7781c0315ec34242bbedc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala @@ -30,7 +30,12 @@ import org.apache.spark.unsafe.types.UTF8String * Returns an Array containing the evaluation of all children expressions. */ @ExpressionDescription( - usage = "_FUNC_(n0, ...) - Returns an array with the given elements.") + usage = "_FUNC_(expr, ...) - Returns an array with the given elements.", + extended = """ + Examples: + > SELECT _FUNC_(1, 2, 3); + [1,2,3] + """) case class CreateArray(children: Seq[Expression]) extends Expression { override def foldable: Boolean = children.forall(_.foldable) @@ -84,7 +89,12 @@ case class CreateArray(children: Seq[Expression]) extends Expression { * The children are a flatted sequence of kv pairs, e.g. (key1, value1, key2, value2, ...) */ @ExpressionDescription( - usage = "_FUNC_(key0, value0, key1, value1...) - Creates a map with the given key/value pairs.") + usage = "_FUNC_(key0, value0, key1, value1, ...) - Creates a map with the given key/value pairs.", + extended = """ + Examples: + > SELECT _FUNC_(1.0, '2', 3.0, '4'); + {1.0:"2",3.0:"4"} + """) case class CreateMap(children: Seq[Expression]) extends Expression { lazy val keys = children.indices.filter(_ % 2 == 0).map(children) lazy val values = children.indices.filter(_ % 2 != 0).map(children) @@ -276,7 +286,12 @@ trait CreateNamedStructLike extends Expression { */ // scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(name1, val1, name2, val2, ...) - Creates a struct with the given field names and values.") + usage = "_FUNC_(name1, val1, name2, val2, ...) - Creates a struct with the given field names and values.", + extended = """ + Examples: + > SELECT _FUNC_("a", 1, "b", 2, "c", 3); + {"a":1,"b":2,"c":3} + """) // scalastyle:on line.size.limit case class CreateNamedStruct(children: Seq[Expression]) extends CreateNamedStructLike { @@ -329,8 +344,12 @@ case class CreateNamedStructUnsafe(children: Seq[Expression]) extends CreateName */ // scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(text[, pairDelim, keyValueDelim]) - Creates a map after splitting the text into key/value pairs using delimiters. Default delimiters are ',' for pairDelim and ':' for keyValueDelim.", - extended = """ > SELECT _FUNC_('a:1,b:2,c:3',',',':');\n map("a":"1","b":"2","c":"3") """) + usage = "_FUNC_(text[, pairDelim[, keyValueDelim]]) - Creates a map after splitting the text into key/value pairs using delimiters. Default delimiters are ',' for `pairDelim` and ':' for `keyValueDelim`.", + extended = """ + Examples: + > SELECT _FUNC_('a:1,b:2,c:3', ',', ':'); + map("a":"1","b":"2","c":"3") + """) // scalastyle:on line.size.limit case class StringToMap(text: Expression, pairDelim: Expression, keyValueDelim: Expression) extends TernaryExpression with CodegenFallback with ExpectsInputTypes { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala index 71d4e9a3c94711dd28935f9c18c82bdd99dfb419..a7d9e2dfcdb62657550b11a4af7bf5acae45fe12 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala @@ -24,7 +24,12 @@ import org.apache.spark.sql.types._ // scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(expr1,expr2,expr3) - If expr1 is TRUE then IF() returns expr2; otherwise it returns expr3.") + usage = "_FUNC_(expr1, expr2, expr3) - If `expr1` evaluates to true, then returns `expr2`; otherwise returns `expr3`.", + extended = """ + Examples: + > SELECT _FUNC_(1 < 2, 'a', 'b'); + a + """) // scalastyle:on line.size.limit case class If(predicate: Expression, trueValue: Expression, falseValue: Expression) extends Expression { @@ -162,7 +167,7 @@ abstract class CaseWhenBase( */ // scalastyle:off line.size.limit @ExpressionDescription( - usage = "CASE WHEN a THEN b [WHEN c THEN d]* [ELSE e] END - When a = true, returns b; when c = true, return d; else return e.") + usage = "CASE WHEN expr1 THEN expr2 [WHEN expr3 THEN expr4]* [ELSE expr5] END - When `expr1` = true, returns `expr2`; when `expr3` = true, return `expr4`; else return `expr5`.") // scalastyle:on line.size.limit case class CaseWhen( val branches: Seq[(Expression, Expression)], diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index 05bfa7dcfc88f9edeab8836ec755fc52751dced2..9cec6be841de026a5a1f34810caa7867e22aec36 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -75,8 +75,12 @@ case class CurrentTimestamp() extends LeafExpression with CodegenFallback { * Adds a number of days to startdate. */ @ExpressionDescription( - usage = "_FUNC_(start_date, num_days) - Returns the date that is num_days after start_date.", - extended = "> SELECT _FUNC_('2016-07-30', 1);\n '2016-07-31'") + usage = "_FUNC_(start_date, num_days) - Returns the date that is `num_days` after `start_date`.", + extended = """ + Examples: + > SELECT _FUNC_('2016-07-30', 1); + 2016-07-31 + """) case class DateAdd(startDate: Expression, days: Expression) extends BinaryExpression with ImplicitCastInputTypes { @@ -104,8 +108,12 @@ case class DateAdd(startDate: Expression, days: Expression) * Subtracts a number of days to startdate. */ @ExpressionDescription( - usage = "_FUNC_(start_date, num_days) - Returns the date that is num_days before start_date.", - extended = "> SELECT _FUNC_('2016-07-30', 1);\n '2016-07-29'") + usage = "_FUNC_(start_date, num_days) - Returns the date that is `num_days` before `start_date`.", + extended = """ + Examples: + > SELECT _FUNC_('2016-07-30', 1); + 2016-07-29 + """) case class DateSub(startDate: Expression, days: Expression) extends BinaryExpression with ImplicitCastInputTypes { override def left: Expression = startDate @@ -129,8 +137,12 @@ case class DateSub(startDate: Expression, days: Expression) } @ExpressionDescription( - usage = "_FUNC_(param) - Returns the hour component of the string/timestamp/interval.", - extended = "> SELECT _FUNC_('2009-07-30 12:58:59');\n 12") + usage = "_FUNC_(timestamp) - Returns the hour component of the string/timestamp.", + extended = """ + Examples: + > SELECT _FUNC_('2009-07-30 12:58:59'); + 12 + """) case class Hour(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) @@ -148,8 +160,12 @@ case class Hour(child: Expression) extends UnaryExpression with ImplicitCastInpu } @ExpressionDescription( - usage = "_FUNC_(param) - Returns the minute component of the string/timestamp/interval.", - extended = "> SELECT _FUNC_('2009-07-30 12:58:59');\n 58") + usage = "_FUNC_(timestamp) - Returns the minute component of the string/timestamp.", + extended = """ + Examples: + > SELECT _FUNC_('2009-07-30 12:58:59'); + 58 + """) case class Minute(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) @@ -167,8 +183,12 @@ case class Minute(child: Expression) extends UnaryExpression with ImplicitCastIn } @ExpressionDescription( - usage = "_FUNC_(param) - Returns the second component of the string/timestamp/interval.", - extended = "> SELECT _FUNC_('2009-07-30 12:58:59');\n 59") + usage = "_FUNC_(timestamp) - Returns the second component of the string/timestamp.", + extended = """ + Examples: + > SELECT _FUNC_('2009-07-30 12:58:59'); + 59 + """) case class Second(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) @@ -186,8 +206,12 @@ case class Second(child: Expression) extends UnaryExpression with ImplicitCastIn } @ExpressionDescription( - usage = "_FUNC_(param) - Returns the day of year of date/timestamp.", - extended = "> SELECT _FUNC_('2016-04-09');\n 100") + usage = "_FUNC_(date) - Returns the day of year of the date/timestamp.", + extended = """ + Examples: + > SELECT _FUNC_('2016-04-09'); + 100 + """) case class DayOfYear(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(DateType) @@ -205,8 +229,12 @@ case class DayOfYear(child: Expression) extends UnaryExpression with ImplicitCas } @ExpressionDescription( - usage = "_FUNC_(param) - Returns the year component of the date/timestamp/interval.", - extended = "> SELECT _FUNC_('2016-07-30');\n 2016") + usage = "_FUNC_(date) - Returns the year component of the date/timestamp.", + extended = """ + Examples: + > SELECT _FUNC_('2016-07-30'); + 2016 + """) case class Year(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(DateType) @@ -224,7 +252,12 @@ case class Year(child: Expression) extends UnaryExpression with ImplicitCastInpu } @ExpressionDescription( - usage = "_FUNC_(param) - Returns the quarter of the year for date, in the range 1 to 4.") + usage = "_FUNC_(date) - Returns the quarter of the year for date, in the range 1 to 4.", + extended = """ + Examples: + > SELECT _FUNC_('2016-08-31'); + 3 + """) case class Quarter(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(DateType) @@ -242,8 +275,12 @@ case class Quarter(child: Expression) extends UnaryExpression with ImplicitCastI } @ExpressionDescription( - usage = "_FUNC_(param) - Returns the month component of the date/timestamp/interval", - extended = "> SELECT _FUNC_('2016-07-30');\n 7") + usage = "_FUNC_(date) - Returns the month component of the date/timestamp.", + extended = """ + Examples: + > SELECT _FUNC_('2016-07-30'); + 7 + """) case class Month(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(DateType) @@ -261,8 +298,12 @@ case class Month(child: Expression) extends UnaryExpression with ImplicitCastInp } @ExpressionDescription( - usage = "_FUNC_(param) - Returns the day of month of date/timestamp, or the day of interval.", - extended = "> SELECT _FUNC_('2009-07-30');\n 30") + usage = "_FUNC_(date) - Returns the day of month of the date/timestamp.", + extended = """ + Examples: + > SELECT _FUNC_('2009-07-30'); + 30 + """) case class DayOfMonth(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(DateType) @@ -280,8 +321,12 @@ case class DayOfMonth(child: Expression) extends UnaryExpression with ImplicitCa } @ExpressionDescription( - usage = "_FUNC_(param) - Returns the week of the year of the given date.", - extended = "> SELECT _FUNC_('2008-02-20');\n 8") + usage = "_FUNC_(date) - Returns the week of the year of the given date.", + extended = """ + Examples: + > SELECT _FUNC_('2008-02-20'); + 8 + """) case class WeekOfYear(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(DateType) @@ -320,8 +365,12 @@ case class WeekOfYear(child: Expression) extends UnaryExpression with ImplicitCa // scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(date/timestamp/string, fmt) - Converts a date/timestamp/string to a value of string in the format specified by the date format fmt.", - extended = "> SELECT _FUNC_('2016-04-08', 'y')\n '2016'") + usage = "_FUNC_(timestamp, fmt) - Converts `timestamp` to a value of string in the format specified by the date format `fmt`.", + extended = """ + Examples: + > SELECT _FUNC_('2016-04-08', 'y'); + 2016 + """) // scalastyle:on line.size.limit case class DateFormatClass(left: Expression, right: Expression) extends BinaryExpression with ImplicitCastInputTypes { @@ -351,7 +400,12 @@ case class DateFormatClass(left: Expression, right: Expression) extends BinaryEx * Deterministic version of [[UnixTimestamp]], must have at least one parameter. */ @ExpressionDescription( - usage = "_FUNC_(date[, pattern]) - Returns the UNIX timestamp of the give time.") + usage = "_FUNC_(expr[, pattern]) - Returns the UNIX timestamp of the give time.", + extended = """ + Examples: + > SELECT _FUNC_('2016-04-08', 'yyyy-MM-dd'); + 1460041200 + """) case class ToUnixTimestamp(timeExp: Expression, format: Expression) extends UnixTime { override def left: Expression = timeExp override def right: Expression = format @@ -374,7 +428,14 @@ case class ToUnixTimestamp(timeExp: Expression, format: Expression) extends Unix * second parameter. */ @ExpressionDescription( - usage = "_FUNC_([date[, pattern]]) - Returns the UNIX timestamp of current or specified time.") + usage = "_FUNC_([expr[, pattern]]) - Returns the UNIX timestamp of current or specified time.", + extended = """ + Examples: + > SELECT _FUNC_(); + 1476884637 + > SELECT _FUNC_('2016-04-08', 'yyyy-MM-dd'); + 1460041200 + """) case class UnixTimestamp(timeExp: Expression, format: Expression) extends UnixTime { override def left: Expression = timeExp override def right: Expression = format @@ -497,8 +558,12 @@ abstract class UnixTime extends BinaryExpression with ExpectsInputTypes { * Note that hive Language Manual says it returns 0 if fail, but in fact it returns null. */ @ExpressionDescription( - usage = "_FUNC_(unix_time, format) - Returns unix_time in the specified format", - extended = "> SELECT _FUNC_(0, 'yyyy-MM-dd HH:mm:ss');\n '1970-01-01 00:00:00'") + usage = "_FUNC_(unix_time, format) - Returns `unix_time` in the specified `format`.", + extended = """ + Examples: + > SELECT _FUNC_(0, 'yyyy-MM-dd HH:mm:ss'); + 1970-01-01 00:00:00 + """) case class FromUnixTime(sec: Expression, format: Expression) extends BinaryExpression with ImplicitCastInputTypes { @@ -586,7 +651,11 @@ case class FromUnixTime(sec: Expression, format: Expression) */ @ExpressionDescription( usage = "_FUNC_(date) - Returns the last day of the month which the date belongs to.", - extended = "> SELECT _FUNC_('2009-01-12');\n '2009-01-31'") + extended = """ + Examples: + > SELECT _FUNC_('2009-01-12'); + 2009-01-31 + """) case class LastDay(startDate: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def child: Expression = startDate @@ -615,8 +684,12 @@ case class LastDay(startDate: Expression) extends UnaryExpression with ImplicitC */ // scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(start_date, day_of_week) - Returns the first date which is later than start_date and named as indicated.", - extended = "> SELECT _FUNC_('2015-01-14', 'TU');\n '2015-01-20'") + usage = "_FUNC_(start_date, day_of_week) - Returns the first date which is later than `start_date` and named as indicated.", + extended = """ + Examples: + > SELECT _FUNC_('2015-01-14', 'TU'); + 2015-01-20 + """) // scalastyle:on line.size.limit case class NextDay(startDate: Expression, dayOfWeek: Expression) extends BinaryExpression with ImplicitCastInputTypes { @@ -701,11 +774,17 @@ case class TimeAdd(start: Expression, interval: Expression) } /** - * Assumes given timestamp is UTC and converts to given timezone. + * Given a timestamp, which corresponds to a certain time of day in UTC, returns another timestamp + * that corresponds to the same time of day in the given timezone. */ // scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(timestamp, string timezone) - Assumes given timestamp is UTC and converts to given timezone.") + usage = "_FUNC_(timestamp, timezone) - Given a timestamp, which corresponds to a certain time of day in UTC, returns another timestamp that corresponds to the same time of day in the given timezone.", + extended = """ + Examples: + > SELECT from_utc_timestamp('2016-08-31', 'Asia/Seoul'); + 2016-08-31 09:00:00 + """) // scalastyle:on line.size.limit case class FromUTCTimestamp(left: Expression, right: Expression) extends BinaryExpression with ImplicitCastInputTypes { @@ -784,9 +863,15 @@ case class TimeSub(start: Expression, interval: Expression) /** * Returns the date that is num_months after start_date. */ +// scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(start_date, num_months) - Returns the date that is num_months after start_date.", - extended = "> SELECT _FUNC_('2016-08-31', 1);\n '2016-09-30'") + usage = "_FUNC_(start_date, num_months) - Returns the date that is `num_months` after `start_date`.", + extended = """ + Examples: + > SELECT _FUNC_('2016-08-31', 1); + 2016-09-30 + """) +// scalastyle:on line.size.limit case class AddMonths(startDate: Expression, numMonths: Expression) extends BinaryExpression with ImplicitCastInputTypes { @@ -814,9 +899,15 @@ case class AddMonths(startDate: Expression, numMonths: Expression) /** * Returns number of months between dates date1 and date2. */ +// scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(date1, date2) - returns number of months between dates date1 and date2.", - extended = "> SELECT _FUNC_('1997-02-28 10:30:00', '1996-10-30');\n 3.94959677") + usage = "_FUNC_(timestamp1, timestamp2) - Returns number of months between `timestamp1` and `timestamp2`.", + extended = """ + Examples: + > SELECT _FUNC_('1997-02-28 10:30:00', '1996-10-30'); + 3.94959677 + """) +// scalastyle:on line.size.limit case class MonthsBetween(date1: Expression, date2: Expression) extends BinaryExpression with ImplicitCastInputTypes { @@ -842,11 +933,17 @@ case class MonthsBetween(date1: Expression, date2: Expression) } /** - * Assumes given timestamp is in given timezone and converts to UTC. + * Given a timestamp, which corresponds to a certain time of day in the given timezone, returns + * another timestamp that corresponds to the same time of day in UTC. */ // scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(timestamp, string timezone) - Assumes given timestamp is in given timezone and converts to UTC.") + usage = "_FUNC_(timestamp, timezone) - Given a timestamp, which corresponds to a certain time of day in the given timezone, returns another timestamp that corresponds to the same time of day in UTC.", + extended = """ + Examples: + > SELECT _FUNC_('2016-08-31', 'Asia/Seoul'); + 2016-08-30 15:00:00 + """) // scalastyle:on line.size.limit case class ToUTCTimestamp(left: Expression, right: Expression) extends BinaryExpression with ImplicitCastInputTypes { @@ -897,8 +994,12 @@ case class ToUTCTimestamp(left: Expression, right: Expression) * Returns the date part of a timestamp or string. */ @ExpressionDescription( - usage = "_FUNC_(expr) - Extracts the date part of the date or datetime expression expr.", - extended = "> SELECT _FUNC_('2009-07-30 04:17:52');\n '2009-07-30'") + usage = "_FUNC_(expr) - Extracts the date part of the date or timestamp expression `expr`.", + extended = """ + Examples: + > SELECT _FUNC_('2009-07-30 04:17:52'); + 2009-07-30 + """) case class ToDate(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { // Implicit casting of spark will accept string in both date and timestamp format, as @@ -921,8 +1022,14 @@ case class ToDate(child: Expression) extends UnaryExpression with ImplicitCastIn */ // scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(date, fmt) - Returns returns date with the time portion of the day truncated to the unit specified by the format model fmt.", - extended = "> SELECT _FUNC_('2009-02-12', 'MM')\n '2009-02-01'\n> SELECT _FUNC_('2015-10-27', 'YEAR');\n '2015-01-01'") + usage = "_FUNC_(date, fmt) - Returns `date` with the time portion of the day truncated to the unit specified by the format model `fmt`.", + extended = """ + Examples: + > SELECT _FUNC_('2009-02-12', 'MM'); + 2009-02-01 + > SELECT _FUNC_('2015-10-27', 'YEAR'); + 2015-01-01 + """) // scalastyle:on line.size.limit case class TruncDate(date: Expression, format: Expression) extends BinaryExpression with ImplicitCastInputTypes { @@ -994,8 +1101,12 @@ case class TruncDate(date: Expression, format: Expression) * Returns the number of days from startDate to endDate. */ @ExpressionDescription( - usage = "_FUNC_(date1, date2) - Returns the number of days between date1 and date2.", - extended = "> SELECT _FUNC_('2009-07-30', '2009-07-31');\n 1") + usage = "_FUNC_(date1, date2) - Returns the number of days between `date1` and `date2`.", + extended = """ + Examples: + > SELECT _FUNC_('2009-07-30', '2009-07-31'); + 1 + """) case class DateDiff(endDate: Expression, startDate: Expression) extends BinaryExpression with ImplicitCastInputTypes { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala index f74208ff66db77006ef50864f6bf92a9c888946e..d042bfb63d567814450ab3521e704e5aa24b5538 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -102,8 +102,13 @@ case class UserDefinedGenerator( * }}} */ @ExpressionDescription( - usage = "_FUNC_(n, v1, ..., vk) - Separate v1, ..., vk into n rows.", - extended = "> SELECT _FUNC_(2, 1, 2, 3);\n [1,2]\n [3,null]") + usage = "_FUNC_(n, expr1, ..., exprk) - Separates `expr1`, ..., `exprk` into `n` rows.", + extended = """ + Examples: + > SELECT _FUNC_(2, 1, 2, 3); + 1 2 + 3 NULL + """) case class Stack(children: Seq[Expression]) extends Expression with Generator with CodegenFallback { @@ -226,8 +231,13 @@ abstract class ExplodeBase(child: Expression, position: Boolean) */ // scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(a) - Separates the elements of array a into multiple rows, or the elements of map a into multiple rows and columns.", - extended = "> SELECT _FUNC_(array(10,20));\n 10\n 20") + usage = "_FUNC_(expr) - Separates the elements of array `expr` into multiple rows, or the elements of map `expr` into multiple rows and columns.", + extended = """ + Examples: + > SELECT _FUNC_(array(10, 20)); + 10 + 20 + """) // scalastyle:on line.size.limit case class Explode(child: Expression) extends ExplodeBase(child, position = false) @@ -242,8 +252,13 @@ case class Explode(child: Expression) extends ExplodeBase(child, position = fals */ // scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(a) - Separates the elements of array a into multiple rows with positions, or the elements of a map into multiple rows and columns with positions.", - extended = "> SELECT _FUNC_(array(10,20));\n 0\t10\n 1\t20") + usage = "_FUNC_(expr) - Separates the elements of array `expr` into multiple rows with positions, or the elements of map `expr` into multiple rows and columns with positions.", + extended = """ + Examples: + > SELECT _FUNC_(array(10,20)); + 0 10 + 1 20 + """) // scalastyle:on line.size.limit case class PosExplode(child: Expression) extends ExplodeBase(child, position = true) @@ -251,8 +266,13 @@ case class PosExplode(child: Expression) extends ExplodeBase(child, position = t * Explodes an array of structs into a table. */ @ExpressionDescription( - usage = "_FUNC_(a) - Explodes an array of structs into a table.", - extended = "> SELECT _FUNC_(array(struct(1, 'a'), struct(2, 'b')));\n [1,a]\n [2,b]") + usage = "_FUNC_(expr) - Explodes an array of structs into a table.", + extended = """ + Examples: + > SELECT _FUNC_(array(struct(1, 'a'), struct(2, 'b'))); + 1 a + 2 b + """) case class Inline(child: Expression) extends UnaryExpression with Generator with CodegenFallback { override def checkInputDataTypes(): TypeCheckResult = child.dataType match { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala index 244a5a34f3594ea25e38f02f8706f505e26041e3..e034735375274d10d27609ccc2659d0667fa03b5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala @@ -110,7 +110,12 @@ private[this] object SharedFactory { * of the extracted json object. It will return null if the input json string is invalid. */ @ExpressionDescription( - usage = "_FUNC_(json_txt, path) - Extract a json object from path") + usage = "_FUNC_(json_txt, path) - Extracts a json object from `path`.", + extended = """ + Examples: + > SELECT _FUNC_('{"a":"b"}', '$.a'); + b + """) case class GetJsonObject(json: Expression, path: Expression) extends BinaryExpression with ExpectsInputTypes with CodegenFallback { @@ -326,7 +331,12 @@ case class GetJsonObject(json: Expression, path: Expression) // scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(jsonStr, p1, p2, ..., pn) - like get_json_object, but it takes multiple names and return a tuple. All the input parameters and output column types are string.") + usage = "_FUNC_(jsonStr, p1, p2, ..., pn) - Return a tuple like the function get_json_object, but it takes multiple names. All the input parameters and output column types are string.", + extended = """ + Examples: + > SELECT _FUNC_('{"a":1, "b":2}', 'a', 'b'); + 1 2 + """) // scalastyle:on line.size.limit case class JsonTuple(children: Seq[Expression]) extends Generator with CodegenFallback { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala index 5152265152aed7921427f07bffafc04b4479124a..a60494a5bb69d89d2ef3180a189cfb446e155c45 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala @@ -139,8 +139,12 @@ abstract class BinaryMathExpression(f: (Double, Double) => Double, name: String) * evaluated by the optimizer during constant folding. */ @ExpressionDescription( - usage = "_FUNC_() - Returns Euler's number, E.", - extended = "> SELECT _FUNC_();\n 2.718281828459045") + usage = "_FUNC_() - Returns Euler's number, e.", + extended = """ + Examples: + > SELECT _FUNC_(); + 2.718281828459045 + """) case class EulerNumber() extends LeafMathExpression(math.E, "E") /** @@ -148,8 +152,12 @@ case class EulerNumber() extends LeafMathExpression(math.E, "E") * evaluated by the optimizer during constant folding. */ @ExpressionDescription( - usage = "_FUNC_() - Returns PI.", - extended = "> SELECT _FUNC_();\n 3.141592653589793") + usage = "_FUNC_() - Returns pi.", + extended = """ + Examples: + > SELECT _FUNC_(); + 3.141592653589793 + """) case class Pi() extends LeafMathExpression(math.Pi, "PI") //////////////////////////////////////////////////////////////////////////////////////////////////// @@ -158,29 +166,61 @@ case class Pi() extends LeafMathExpression(math.Pi, "PI") //////////////////////////////////////////////////////////////////////////////////////////////////// //////////////////////////////////////////////////////////////////////////////////////////////////// +// scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(x) - Returns the arc cosine of x if -1<=x<=1 or NaN otherwise.", - extended = "> SELECT _FUNC_(1);\n 0.0\n> SELECT _FUNC_(2);\n NaN") + usage = "_FUNC_(expr) - Returns the inverse cosine (a.k.a. arccosine) of `expr` if -1<=`expr`<=1 or NaN otherwise.", + extended = """ + Examples: + > SELECT _FUNC_(1); + 0.0 + > SELECT _FUNC_(2); + NaN + """) +// scalastyle:on line.size.limit case class Acos(child: Expression) extends UnaryMathExpression(math.acos, "ACOS") +// scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(x) - Returns the arc sin of x if -1<=x<=1 or NaN otherwise.", - extended = "> SELECT _FUNC_(0);\n 0.0\n> SELECT _FUNC_(2);\n NaN") + usage = "_FUNC_(expr) - Returns the inverse sine (a.k.a. arcsine) the arc sin of `expr` if -1<=`expr`<=1 or NaN otherwise.", + extended = """ + Examples: + > SELECT _FUNC_(0); + 0.0 + > SELECT _FUNC_(2); + NaN + """) +// scalastyle:on line.size.limit case class Asin(child: Expression) extends UnaryMathExpression(math.asin, "ASIN") +// scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(x) - Returns the arc tangent.", - extended = "> SELECT _FUNC_(0);\n 0.0") + usage = "_FUNC_(expr) - Returns the inverse tangent (a.k.a. arctangent).", + extended = """ + Examples: + > SELECT _FUNC_(0); + 0.0 + """) +// scalastyle:on line.size.limit case class Atan(child: Expression) extends UnaryMathExpression(math.atan, "ATAN") @ExpressionDescription( - usage = "_FUNC_(x) - Returns the cube root of a double value.", - extended = "> SELECT _FUNC_(27.0);\n 3.0") + usage = "_FUNC_(expr) - Returns the cube root of `expr`.", + extended = """ + Examples: + > SELECT _FUNC_(27.0); + 3.0 + """) case class Cbrt(child: Expression) extends UnaryMathExpression(math.cbrt, "CBRT") @ExpressionDescription( - usage = "_FUNC_(x) - Returns the smallest integer not smaller than x.", - extended = "> SELECT _FUNC_(-0.1);\n 0\n> SELECT _FUNC_(5);\n 5") + usage = "_FUNC_(expr) - Returns the smallest integer not smaller than `expr`.", + extended = """ + Examples: + > SELECT _FUNC_(-0.1); + 0 + > SELECT _FUNC_(5); + 5 + """) case class Ceil(child: Expression) extends UnaryMathExpression(math.ceil, "CEIL") { override def dataType: DataType = child.dataType match { case dt @ DecimalType.Fixed(_, 0) => dt @@ -208,13 +248,21 @@ case class Ceil(child: Expression) extends UnaryMathExpression(math.ceil, "CEIL" } @ExpressionDescription( - usage = "_FUNC_(x) - Returns the cosine of x.", - extended = "> SELECT _FUNC_(0);\n 1.0") + usage = "_FUNC_(expr) - Returns the cosine of `expr`.", + extended = """ + Examples: + > SELECT _FUNC_(0); + 1.0 + """) case class Cos(child: Expression) extends UnaryMathExpression(math.cos, "COS") @ExpressionDescription( - usage = "_FUNC_(x) - Returns the hyperbolic cosine of x.", - extended = "> SELECT _FUNC_(0);\n 1.0") + usage = "_FUNC_(expr) - Returns the hyperbolic cosine of `expr`.", + extended = """ + Examples: + > SELECT _FUNC_(0); + 1.0 + """) case class Cosh(child: Expression) extends UnaryMathExpression(math.cosh, "COSH") /** @@ -225,8 +273,14 @@ case class Cosh(child: Expression) extends UnaryMathExpression(math.cosh, "COSH" * @param toBaseExpr to which base */ @ExpressionDescription( - usage = "_FUNC_(num, from_base, to_base) - Convert num from from_base to to_base.", - extended = "> SELECT _FUNC_('100', 2, 10);\n '4'\n> SELECT _FUNC_(-10, 16, -10);\n '16'") + usage = "_FUNC_(num, from_base, to_base) - Convert `num` from `from_base` to `to_base`.", + extended = """ + Examples: + > SELECT _FUNC_('100', 2, 10); + 4 + > SELECT _FUNC_(-10, 16, -10); + 16 + """) case class Conv(numExpr: Expression, fromBaseExpr: Expression, toBaseExpr: Expression) extends TernaryExpression with ImplicitCastInputTypes { @@ -256,18 +310,32 @@ case class Conv(numExpr: Expression, fromBaseExpr: Expression, toBaseExpr: Expre } @ExpressionDescription( - usage = "_FUNC_(x) - Returns e to the power of x.", - extended = "> SELECT _FUNC_(0);\n 1.0") + usage = "_FUNC_(expr) - Returns e to the power of `expr`.", + extended = """ + Examples: + > SELECT _FUNC_(0); + 1.0 + """) case class Exp(child: Expression) extends UnaryMathExpression(math.exp, "EXP") @ExpressionDescription( - usage = "_FUNC_(x) - Returns exp(x) - 1.", - extended = "> SELECT _FUNC_(0);\n 0.0") + usage = "_FUNC_(expr) - Returns exp(`expr`) - 1.", + extended = """ + Examples: + > SELECT _FUNC_(0); + 0.0 + """) case class Expm1(child: Expression) extends UnaryMathExpression(math.expm1, "EXPM1") @ExpressionDescription( - usage = "_FUNC_(x) - Returns the largest integer not greater than x.", - extended = "> SELECT _FUNC_(-0.1);\n -1\n> SELECT _FUNC_(5);\n 5") + usage = "_FUNC_(expr) - Returns the largest integer not greater than `expr`.", + extended = """ + Examples: + > SELECT _FUNC_(-0.1); + -1 + > SELECT _FUNC_(5); + 5 + """) case class Floor(child: Expression) extends UnaryMathExpression(math.floor, "FLOOR") { override def dataType: DataType = child.dataType match { case dt @ DecimalType.Fixed(_, 0) => dt @@ -326,8 +394,12 @@ object Factorial { } @ExpressionDescription( - usage = "_FUNC_(n) - Returns n factorial for n is [0..20]. Otherwise, NULL.", - extended = "> SELECT _FUNC_(5);\n 120") + usage = "_FUNC_(expr) - Returns the factorial of `expr`. `expr` is [0..20]. Otherwise, null.", + extended = """ + Examples: + > SELECT _FUNC_(5); + 120 + """) case class Factorial(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def inputTypes: Seq[DataType] = Seq(IntegerType) @@ -361,13 +433,21 @@ case class Factorial(child: Expression) extends UnaryExpression with ImplicitCas } @ExpressionDescription( - usage = "_FUNC_(x) - Returns the natural logarithm of x with base e.", - extended = "> SELECT _FUNC_(1);\n 0.0") + usage = "_FUNC_(expr) - Returns the natural logarithm (base e) of `expr`.", + extended = """ + Examples: + > SELECT _FUNC_(1); + 0.0 + """) case class Log(child: Expression) extends UnaryLogExpression(math.log, "LOG") @ExpressionDescription( - usage = "_FUNC_(x) - Returns the logarithm of x with base 2.", - extended = "> SELECT _FUNC_(2);\n 1.0") + usage = "_FUNC_(expr) - Returns the logarithm of `expr` with base 2.", + extended = """ + Examples: + > SELECT _FUNC_(2); + 1.0 + """) case class Log2(child: Expression) extends UnaryLogExpression((x: Double) => math.log(x) / math.log(2), "LOG2") { override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { @@ -384,71 +464,127 @@ case class Log2(child: Expression) } @ExpressionDescription( - usage = "_FUNC_(x) - Returns the logarithm of x with base 10.", - extended = "> SELECT _FUNC_(10);\n 1.0") + usage = "_FUNC_(expr) - Returns the logarithm of `expr` with base 10.", + extended = """ + Examples: + > SELECT _FUNC_(10); + 1.0 + """) case class Log10(child: Expression) extends UnaryLogExpression(math.log10, "LOG10") @ExpressionDescription( - usage = "_FUNC_(x) - Returns log(1 + x).", - extended = "> SELECT _FUNC_(0);\n 0.0") + usage = "_FUNC_(expr) - Returns log(1 + `expr`).", + extended = """ + Examples: + > SELECT _FUNC_(0); + 0.0 + """) case class Log1p(child: Expression) extends UnaryLogExpression(math.log1p, "LOG1P") { protected override val yAsymptote: Double = -1.0 } +// scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(x, d) - Return the rounded x at d decimal places.", - extended = "> SELECT _FUNC_(12.3456, 1);\n 12.3") + usage = "_FUNC_(expr) - Returns the double value that is closest in value to the argument and is equal to a mathematical integer.", + extended = """ + Examples: + > SELECT _FUNC_(12.3456); + 12.0 + """) +// scalastyle:on line.size.limit case class Rint(child: Expression) extends UnaryMathExpression(math.rint, "ROUND") { override def funcName: String = "rint" } @ExpressionDescription( - usage = "_FUNC_(x) - Returns the sign of x.", - extended = "> SELECT _FUNC_(40);\n 1.0") + usage = "_FUNC_(expr) - Returns -1.0, 0.0 or 1.0 as `expr` is negative, 0 or positive.", + extended = """ + Examples: + > SELECT _FUNC_(40); + 1.0 + """) case class Signum(child: Expression) extends UnaryMathExpression(math.signum, "SIGNUM") @ExpressionDescription( - usage = "_FUNC_(x) - Returns the sine of x.", - extended = "> SELECT _FUNC_(0);\n 0.0") + usage = "_FUNC_(expr) - Returns the sine of `expr`.", + extended = """ + Examples: + > SELECT _FUNC_(0); + 0.0 + """) case class Sin(child: Expression) extends UnaryMathExpression(math.sin, "SIN") @ExpressionDescription( - usage = "_FUNC_(x) - Returns the hyperbolic sine of x.", - extended = "> SELECT _FUNC_(0);\n 0.0") + usage = "_FUNC_(expr) - Returns the hyperbolic sine of `expr`.", + extended = """ + Examples: + > SELECT _FUNC_(0); + 0.0 + """) case class Sinh(child: Expression) extends UnaryMathExpression(math.sinh, "SINH") @ExpressionDescription( - usage = "_FUNC_(x) - Returns the square root of x.", - extended = "> SELECT _FUNC_(4);\n 2.0") + usage = "_FUNC_(expr) - Returns the square root of `expr`.", + extended = """ + Examples: + > SELECT _FUNC_(4); + 2.0 + """) case class Sqrt(child: Expression) extends UnaryMathExpression(math.sqrt, "SQRT") @ExpressionDescription( - usage = "_FUNC_(x) - Returns the tangent of x.", - extended = "> SELECT _FUNC_(0);\n 0.0") + usage = "_FUNC_(expr) - Returns the tangent of `expr`.", + extended = """ + Examples: + > SELECT _FUNC_(0); + 0.0 + """) case class Tan(child: Expression) extends UnaryMathExpression(math.tan, "TAN") @ExpressionDescription( - usage = "_FUNC_(x) - Returns the hyperbolic tangent of x.", - extended = "> SELECT _FUNC_(0);\n 0.0") + usage = "_FUNC_(expr) - Returns the hyperbolic tangent of `expr`.", + extended = """ + Examples: + > SELECT _FUNC_(0); + 0.0 + """) case class Tanh(child: Expression) extends UnaryMathExpression(math.tanh, "TANH") @ExpressionDescription( - usage = "_FUNC_(x) - Converts radians to degrees.", - extended = "> SELECT _FUNC_(3.141592653589793);\n 180.0") + usage = "_FUNC_(expr) - Converts radians to degrees.", + extended = """ + Examples: + > SELECT _FUNC_(3.141592653589793); + 180.0 + """) case class ToDegrees(child: Expression) extends UnaryMathExpression(math.toDegrees, "DEGREES") { override def funcName: String = "toDegrees" } @ExpressionDescription( - usage = "_FUNC_(x) - Converts degrees to radians.", - extended = "> SELECT _FUNC_(180);\n 3.141592653589793") + usage = "_FUNC_(expr) - Converts degrees to radians.", + extended = """ + Examples: + > SELECT _FUNC_(180); + 3.141592653589793 + """) case class ToRadians(child: Expression) extends UnaryMathExpression(math.toRadians, "RADIANS") { override def funcName: String = "toRadians" } +// scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(x) - Returns x in binary.", - extended = "> SELECT _FUNC_(13);\n '1101'") + usage = "_FUNC_(expr) - Returns the string representation of the long value `expr` represented in binary.", + extended = """ + Examples: + > SELECT _FUNC_(13); + 1101 + > SELECT _FUNC_(-13); + 1111111111111111111111111111111111111111111111111111111111110011 + > SELECT _FUNC_(13.3); + 1101 + """) +// scalastyle:on line.size.limit case class Bin(child: Expression) extends UnaryExpression with Serializable with ImplicitCastInputTypes { @@ -541,8 +677,14 @@ object Hex { * and returns the resulting STRING. Negative numbers would be treated as two's complement. */ @ExpressionDescription( - usage = "_FUNC_(x) - Convert the argument to hexadecimal.", - extended = "> SELECT _FUNC_(17);\n '11'\n> SELECT _FUNC_('Spark SQL');\n '537061726B2053514C'") + usage = "_FUNC_(expr) - Converts `expr` to hexadecimal.", + extended = """ + Examples: + > SELECT _FUNC_(17); + 11 + > SELECT _FUNC_('Spark SQL'); + 537061726B2053514C + """) case class Hex(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def inputTypes: Seq[AbstractDataType] = @@ -572,8 +714,12 @@ case class Hex(child: Expression) extends UnaryExpression with ImplicitCastInput * Resulting characters are returned as a byte array. */ @ExpressionDescription( - usage = "_FUNC_(x) - Converts hexadecimal argument to binary.", - extended = "> SELECT decode(_FUNC_('537061726B2053514C'),'UTF-8');\n 'Spark SQL'") + usage = "_FUNC_(expr) - Converts hexadecimal `expr` to binary.", + extended = """ + Examples: + > SELECT decode(_FUNC_('537061726B2053514C'), 'UTF-8'); + Spark SQL + """) case class Unhex(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(StringType) @@ -602,9 +748,15 @@ case class Unhex(child: Expression) extends UnaryExpression with ImplicitCastInp //////////////////////////////////////////////////////////////////////////////////////////////////// //////////////////////////////////////////////////////////////////////////////////////////////////// +// scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(x,y) - Returns the arc tangent2.", - extended = "> SELECT _FUNC_(0, 0);\n 0.0") + usage = "_FUNC_(expr1, expr2) - Returns the angle in radians between the positive x-axis of a plane and the point given by the coordinates (`expr1`, `expr2`).", + extended = """ + Examples: + > SELECT _FUNC_(0, 0); + 0.0 + """) +// scalastyle:on line.size.limit case class Atan2(left: Expression, right: Expression) extends BinaryMathExpression(math.atan2, "ATAN2") { @@ -619,8 +771,12 @@ case class Atan2(left: Expression, right: Expression) } @ExpressionDescription( - usage = "_FUNC_(x1, x2) - Raise x1 to the power of x2.", - extended = "> SELECT _FUNC_(2, 3);\n 8.0") + usage = "_FUNC_(expr1, expr2) - Raises `expr1` to the power of `expr2`.", + extended = """ + Examples: + > SELECT _FUNC_(2, 3); + 8.0 + """) case class Pow(left: Expression, right: Expression) extends BinaryMathExpression(math.pow, "POWER") { override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { @@ -636,8 +792,12 @@ case class Pow(left: Expression, right: Expression) * @param right number of bits to left shift. */ @ExpressionDescription( - usage = "_FUNC_(a, b) - Bitwise left shift.", - extended = "> SELECT _FUNC_(2, 1);\n 4") + usage = "_FUNC_(base, expr) - Bitwise left shift.", + extended = """ + Examples: + > SELECT _FUNC_(2, 1); + 4 + """) case class ShiftLeft(left: Expression, right: Expression) extends BinaryExpression with ImplicitCastInputTypes { @@ -660,14 +820,18 @@ case class ShiftLeft(left: Expression, right: Expression) /** - * Bitwise right shift. + * Bitwise (signed) right shift. * * @param left the base number to shift. * @param right number of bits to right shift. */ @ExpressionDescription( - usage = "_FUNC_(a, b) - Bitwise right shift.", - extended = "> SELECT _FUNC_(4, 1);\n 2") + usage = "_FUNC_(base, expr) - Bitwise (signed) right shift.", + extended = """ + Examples: + > SELECT _FUNC_(4, 1); + 2 + """) case class ShiftRight(left: Expression, right: Expression) extends BinaryExpression with ImplicitCastInputTypes { @@ -696,8 +860,12 @@ case class ShiftRight(left: Expression, right: Expression) * @param right the number of bits to right shift. */ @ExpressionDescription( - usage = "_FUNC_(a, b) - Bitwise unsigned right shift.", - extended = "> SELECT _FUNC_(4, 1);\n 2") + usage = "_FUNC_(base, expr) - Bitwise unsigned right shift.", + extended = """ + Examples: + > SELECT _FUNC_(4, 1); + 2 + """) case class ShiftRightUnsigned(left: Expression, right: Expression) extends BinaryExpression with ImplicitCastInputTypes { @@ -719,8 +887,12 @@ case class ShiftRightUnsigned(left: Expression, right: Expression) } @ExpressionDescription( - usage = "_FUNC_(a, b) - Returns sqrt(a**2 + b**2).", - extended = "> SELECT _FUNC_(3, 4);\n 5.0") + usage = "_FUNC_(expr1, expr2) - Returns sqrt(`expr1`**2 + `expr2`**2).", + extended = """ + Examples: + > SELECT _FUNC_(3, 4); + 5.0 + """) case class Hypot(left: Expression, right: Expression) extends BinaryMathExpression(math.hypot, "HYPOT") @@ -732,8 +904,12 @@ case class Hypot(left: Expression, right: Expression) * @param right the number to compute the logarithm of. */ @ExpressionDescription( - usage = "_FUNC_(b, x) - Returns the logarithm of x with base b.", - extended = "> SELECT _FUNC_(10, 100);\n 2.0") + usage = "_FUNC_(base, expr) - Returns the logarithm of `expr` with `base`.", + extended = """ + Examples: + > SELECT _FUNC_(10, 100); + 2.0 + """) case class Logarithm(left: Expression, right: Expression) extends BinaryMathExpression((c1, c2) => math.log(c2) / math.log(c1), "LOG") { @@ -956,9 +1132,15 @@ abstract class RoundBase(child: Expression, scale: Expression, * Round an expression to d decimal places using HALF_UP rounding mode. * round(2.5) == 3.0, round(3.5) == 4.0. */ +// scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(x, d) - Round x to d decimal places using HALF_UP rounding mode.", - extended = "> SELECT _FUNC_(2.5, 0);\n 3.0") + usage = "_FUNC_(expr, d) - Returns `expr` rounded to `d` decimal places using HALF_UP rounding mode.", + extended = """ + Examples: + > SELECT _FUNC_(2.5, 0); + 3.0 + """) +// scalastyle:on line.size.limit case class Round(child: Expression, scale: Expression) extends RoundBase(child, scale, BigDecimal.RoundingMode.HALF_UP, "ROUND_HALF_UP") with Serializable with ImplicitCastInputTypes { @@ -970,9 +1152,15 @@ case class Round(child: Expression, scale: Expression) * also known as Gaussian rounding or bankers' rounding. * round(2.5) = 2.0, round(3.5) = 4.0. */ +// scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(x, d) - Round x to d decimal places using HALF_EVEN rounding mode.", - extended = "> SELECT _FUNC_(2.5, 0);\n 2.0") + usage = "_FUNC_(expr, d) - Returns `expr` rounded to `d` decimal places using HALF_EVEN rounding mode.", + extended = """ + Examples: + > SELECT _FUNC_(2.5, 0); + 2.0 + """) +// scalastyle:on line.size.limit case class BRound(child: Expression, scale: Expression) extends RoundBase(child, scale, BigDecimal.RoundingMode.HALF_EVEN, "ROUND_HALF_EVEN") with Serializable with ImplicitCastInputTypes { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala index 5ead16908732f1d8bd66d530c8c67001a6675cbe..2ce10ef13215e8977c5b39ed002a0b5f823baa55 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala @@ -38,8 +38,12 @@ import org.apache.spark.unsafe.Platform * For input of type [[BinaryType]] */ @ExpressionDescription( - usage = "_FUNC_(input) - Returns an MD5 128-bit checksum as a hex string of the input", - extended = "> SELECT _FUNC_('Spark');\n '8cde774d6f7333752ed72cacddb05126'") + usage = "_FUNC_(expr) - Returns an MD5 128-bit checksum as a hex string of `expr`.", + extended = """ + Examples: + > SELECT _FUNC_('Spark'); + 8cde774d6f7333752ed72cacddb05126 + """) case class Md5(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def dataType: DataType = StringType @@ -65,10 +69,15 @@ case class Md5(child: Expression) extends UnaryExpression with ImplicitCastInput */ // scalastyle:off line.size.limit @ExpressionDescription( - usage = """_FUNC_(input, bitLength) - Returns a checksum of SHA-2 family as a hex string of the input. - SHA-224, SHA-256, SHA-384, and SHA-512 are supported. Bit length of 0 is equivalent to 256.""", - extended = """> SELECT _FUNC_('Spark', 0); - '529bc3b07127ecb7e53a4dcf1991d9152c24537d919178022b2c42657f79a26b'""") + usage = """ + _FUNC_(expr, bitLength) - Returns a checksum of SHA-2 family as a hex string of `expr`. + SHA-224, SHA-256, SHA-384, and SHA-512 are supported. Bit length of 0 is equivalent to 256. + """, + extended = """ + Examples: + > SELECT _FUNC_('Spark', 256); + 529bc3b07127ecb7e53a4dcf1991d9152c24537d919178022b2c42657f79a26b + """) // scalastyle:on line.size.limit case class Sha2(left: Expression, right: Expression) extends BinaryExpression with Serializable with ImplicitCastInputTypes { @@ -136,8 +145,12 @@ case class Sha2(left: Expression, right: Expression) * For input of type [[BinaryType]] or [[StringType]] */ @ExpressionDescription( - usage = "_FUNC_(input) - Returns a sha1 hash value as a hex string of the input", - extended = "> SELECT _FUNC_('Spark');\n '85f5955f4b27a9a4c2aab6ffe5d7189fc298b92c'") + usage = "_FUNC_(expr) - Returns a sha1 hash value as a hex string of the `expr`.", + extended = """ + Examples: + > SELECT _FUNC_('Spark'); + 85f5955f4b27a9a4c2aab6ffe5d7189fc298b92c + """) case class Sha1(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def dataType: DataType = StringType @@ -159,8 +172,12 @@ case class Sha1(child: Expression) extends UnaryExpression with ImplicitCastInpu * For input of type [[BinaryType]] */ @ExpressionDescription( - usage = "_FUNC_(input) - Returns a cyclic redundancy check value as a bigint of the input", - extended = "> SELECT _FUNC_('Spark');\n '1557323817'") + usage = "_FUNC_(expr) - Returns a cyclic redundancy check value of the `expr` as a bigint.", + extended = """ + Examples: + > SELECT _FUNC_('Spark'); + 1557323817 + """) case class Crc32(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def dataType: DataType = LongType @@ -490,7 +507,12 @@ abstract class InterpretedHashFunction { * and bucketing have same data distribution. */ @ExpressionDescription( - usage = "_FUNC_(a1, a2, ...) - Returns a hash value of the arguments.") + usage = "_FUNC_(expr1, expr2, ...) - Returns a hash value of the arguments.", + extended = """ + Examples: + > SELECT _FUNC_('Spark', array(123), 2); + -1321691492 + """) case class Murmur3Hash(children: Seq[Expression], seed: Int) extends HashExpression[Int] { def this(arguments: Seq[Expression]) = this(arguments, 42) @@ -544,7 +566,12 @@ case class PrintToStderr(child: Expression) extends UnaryExpression { * A function throws an exception if 'condition' is not true. */ @ExpressionDescription( - usage = "_FUNC_(condition) - Throw an exception if 'condition' is not true.") + usage = "_FUNC_(expr) - Throws an exception if `expr` is not true.", + extended = """ + Examples: + > SELECT _FUNC_(0 < 1); + NULL + """) case class AssertTrue(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def nullable: Boolean = true @@ -613,7 +640,11 @@ object XxHash64Function extends InterpretedHashFunction { */ @ExpressionDescription( usage = "_FUNC_() - Returns the current database.", - extended = "> SELECT _FUNC_()") + extended = """ + Examples: + > SELECT _FUNC_(); + default + """) case class CurrentDatabase() extends LeafExpression with Unevaluable { override def dataType: DataType = StringType override def foldable: Boolean = true @@ -631,7 +662,7 @@ case class CurrentDatabase() extends LeafExpression with Unevaluable { * TODO: Support Decimal and date related types */ @ExpressionDescription( - usage = "_FUNC_(a1, a2, ...) - Returns a hash value of the arguments.") + usage = "_FUNC_(expr1, expr2, ...) - Returns a hash value of the arguments.") case class HiveHash(children: Seq[Expression]) extends HashExpression[Int] { override val seed = 0 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala index 70862a87ef9c6ef649dada22c2ce48db06e8019e..8b2e8f3e7ef73ce5505f3a27f0e6c8d4594ef586 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala @@ -34,9 +34,15 @@ import org.apache.spark.sql.types._ * coalesce(null, null, null) => null * }}} */ +// scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(a1, a2, ...) - Returns the first non-null argument if exists. Otherwise, NULL.", - extended = "> SELECT _FUNC_(NULL, 1, NULL);\n 1") + usage = "_FUNC_(expr1, expr2, ...) - Returns the first non-null argument if exists. Otherwise, null.", + extended = """ + Examples: + > SELECT _FUNC_(NULL, 1, NULL); + 1 + """) +// scalastyle:on line.size.limit case class Coalesce(children: Seq[Expression]) extends Expression { /** Coalesce is nullable if all of its children are nullable, or if it has no children. */ @@ -88,7 +94,13 @@ case class Coalesce(children: Seq[Expression]) extends Expression { } -@ExpressionDescription(usage = "_FUNC_(a,b) - Returns b if a is null, or a otherwise.") +@ExpressionDescription( + usage = "_FUNC_(expr1, expr2) - Returns `expr2` if `expr1` is null, or `expr1` otherwise.", + extended = """ + Examples: + > SELECT _FUNC_(NULL, array('2')); + ["2"] + """) case class IfNull(left: Expression, right: Expression, child: Expression) extends RuntimeReplaceable { @@ -101,7 +113,13 @@ case class IfNull(left: Expression, right: Expression, child: Expression) } -@ExpressionDescription(usage = "_FUNC_(a,b) - Returns null if a equals to b, or a otherwise.") +@ExpressionDescription( + usage = "_FUNC_(expr1, expr2) - Returns null if `expr1` equals to `expr2`, or `expr1` otherwise.", + extended = """ + Examples: + > SELECT _FUNC_(2, 2); + NULL + """) case class NullIf(left: Expression, right: Expression, child: Expression) extends RuntimeReplaceable { @@ -114,7 +132,13 @@ case class NullIf(left: Expression, right: Expression, child: Expression) } -@ExpressionDescription(usage = "_FUNC_(a,b) - Returns b if a is null, or a otherwise.") +@ExpressionDescription( + usage = "_FUNC_(expr1, expr2) - Returns `expr2` if `expr1` is null, or `expr1` otherwise.", + extended = """ + Examples: + > SELECT _FUNC_(NULL, array('2')); + ["2"] + """) case class Nvl(left: Expression, right: Expression, child: Expression) extends RuntimeReplaceable { def this(left: Expression, right: Expression) = { @@ -126,7 +150,15 @@ case class Nvl(left: Expression, right: Expression, child: Expression) extends R } -@ExpressionDescription(usage = "_FUNC_(a,b,c) - Returns b if a is not null, or c otherwise.") +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = "_FUNC_(expr1, expr2, expr3) - Returns `expr2` if `expr1` is not null, or `expr3` otherwise.", + extended = """ + Examples: + > SELECT _FUNC_(NULL, 2, 1); + 1 + """) +// scalastyle:on line.size.limit case class Nvl2(expr1: Expression, expr2: Expression, expr3: Expression, child: Expression) extends RuntimeReplaceable { @@ -143,7 +175,12 @@ case class Nvl2(expr1: Expression, expr2: Expression, expr3: Expression, child: * Evaluates to `true` iff it's NaN. */ @ExpressionDescription( - usage = "_FUNC_(a) - Returns true if a is NaN and false otherwise.") + usage = "_FUNC_(expr) - Returns true if `expr` is NaN, or false otherwise.", + extended = """ + Examples: + > SELECT _FUNC_(cast('NaN' as double)); + true + """) case class IsNaN(child: Expression) extends UnaryExpression with Predicate with ImplicitCastInputTypes { @@ -181,7 +218,12 @@ case class IsNaN(child: Expression) extends UnaryExpression * This Expression is useful for mapping NaN values to null. */ @ExpressionDescription( - usage = "_FUNC_(a,b) - Returns a iff it's not NaN, or b otherwise.") + usage = "_FUNC_(expr1, expr2) - Returns `expr1` if it's not NaN, or `expr2` otherwise.", + extended = """ + Examples: + > SELECT _FUNC_(cast('NaN' as double), 123); + 123.0 + """) case class NaNvl(left: Expression, right: Expression) extends BinaryExpression with ImplicitCastInputTypes { @@ -236,7 +278,12 @@ case class NaNvl(left: Expression, right: Expression) * An expression that is evaluated to true if the input is null. */ @ExpressionDescription( - usage = "_FUNC_(a) - Returns true if a is NULL and false otherwise.") + usage = "_FUNC_(expr) - Returns true if `expr` is null, or false otherwise.", + extended = """ + Examples: + > SELECT _FUNC_(1); + false + """) case class IsNull(child: Expression) extends UnaryExpression with Predicate { override def nullable: Boolean = false @@ -257,7 +304,12 @@ case class IsNull(child: Expression) extends UnaryExpression with Predicate { * An expression that is evaluated to true if the input is not null. */ @ExpressionDescription( - usage = "_FUNC_(a) - Returns true if a is not NULL and false otherwise.") + usage = "_FUNC_(expr) - Returns true if `expr` is not null, or false otherwise.", + extended = """ + Examples: + > SELECT _FUNC_(1); + true + """) case class IsNotNull(child: Expression) extends UnaryExpression with Predicate { override def nullable: Boolean = false diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index c941a576d00d69fe1d1e59337f3ef72af89137e8..7946c201f4ffc20bb4e46ee5d83091e58189a612 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -89,7 +89,7 @@ trait PredicateHelper { } @ExpressionDescription( - usage = "_FUNC_ a - Logical not") + usage = "_FUNC_ expr - Logical not.") case class Not(child: Expression) extends UnaryExpression with Predicate with ImplicitCastInputTypes with NullIntolerant { @@ -111,7 +111,7 @@ case class Not(child: Expression) * Evaluates to `true` if `list` contains `value`. */ @ExpressionDescription( - usage = "expr _FUNC_(val1, val2, ...) - Returns true if expr equals to any valN.") + usage = "expr1 _FUNC_(expr2, expr3, ...) - Returns true if `expr` equals to any valN.") case class In(value: Expression, list: Seq[Expression]) extends Predicate with ImplicitCastInputTypes { @@ -248,7 +248,7 @@ case class InSet(child: Expression, hset: Set[Any]) extends UnaryExpression with } @ExpressionDescription( - usage = "a _FUNC_ b - Logical AND.") + usage = "expr1 _FUNC_ expr2 - Logical AND.") case class And(left: Expression, right: Expression) extends BinaryOperator with Predicate { override def inputType: AbstractDataType = BooleanType @@ -311,7 +311,7 @@ case class And(left: Expression, right: Expression) extends BinaryOperator with } @ExpressionDescription( - usage = "a _FUNC_ b - Logical OR.") + usage = "expr1 _FUNC_ expr2 - Logical OR.") case class Or(left: Expression, right: Expression) extends BinaryOperator with Predicate { override def inputType: AbstractDataType = BooleanType @@ -406,7 +406,7 @@ object Equality { } @ExpressionDescription( - usage = "a _FUNC_ b - Returns TRUE if a equals b and false otherwise.") + usage = "expr1 _FUNC_ expr2 - Returns true if `expr1` equals `expr2`, or false otherwise.") case class EqualTo(left: Expression, right: Expression) extends BinaryComparison with NullIntolerant { @@ -432,8 +432,10 @@ case class EqualTo(left: Expression, right: Expression) } @ExpressionDescription( - usage = """a _FUNC_ b - Returns same result with EQUAL(=) operator for non-null operands, - but returns TRUE if both are NULL, FALSE if one of the them is NULL.""") + usage = """ + expr1 _FUNC_ expr2 - Returns same result as the EQUAL(=) operator for non-null operands, + but returns true if both are null, false if one of the them is null. + """) case class EqualNullSafe(left: Expression, right: Expression) extends BinaryComparison { override def inputType: AbstractDataType = AnyDataType @@ -473,7 +475,7 @@ case class EqualNullSafe(left: Expression, right: Expression) extends BinaryComp } @ExpressionDescription( - usage = "a _FUNC_ b - Returns TRUE if a is less than b.") + usage = "expr1 _FUNC_ expr2 - Returns true if `expr1` is less than `expr2`.") case class LessThan(left: Expression, right: Expression) extends BinaryComparison with NullIntolerant { @@ -487,7 +489,7 @@ case class LessThan(left: Expression, right: Expression) } @ExpressionDescription( - usage = "a _FUNC_ b - Returns TRUE if a is not greater than b.") + usage = "expr1 _FUNC_ expr2 - Returns true if `expr1` is less than or equal to `expr2`.") case class LessThanOrEqual(left: Expression, right: Expression) extends BinaryComparison with NullIntolerant { @@ -501,7 +503,7 @@ case class LessThanOrEqual(left: Expression, right: Expression) } @ExpressionDescription( - usage = "a _FUNC_ b - Returns TRUE if a is greater than b.") + usage = "expr1 _FUNC_ expr2 - Returns true if `expr1` is greater than `expr2`.") case class GreaterThan(left: Expression, right: Expression) extends BinaryComparison with NullIntolerant { @@ -515,7 +517,7 @@ case class GreaterThan(left: Expression, right: Expression) } @ExpressionDescription( - usage = "a _FUNC_ b - Returns TRUE if a is not smaller than b.") + usage = "expr1 _FUNC_ expr2 - Returns true if `expr1` is greater than or equal to `expr2`.") case class GreaterThanOrEqual(left: Expression, right: Expression) extends BinaryComparison with NullIntolerant { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala index e09029f5aab9b30d94ac92d3b04b7ccd166ca61b..a331a5557b45530137bd07865f94a48918e222ad 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala @@ -55,8 +55,17 @@ abstract class RDG extends LeafExpression with Nondeterministic { } /** Generate a random column with i.i.d. uniformly distributed values in [0, 1). */ +// scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(a) - Returns a random column with i.i.d. uniformly distributed values in [0, 1).") + usage = "_FUNC_([seed]) - Returns a random value with independent and identically distributed (i.i.d.) uniformly distributed values in [0, 1).", + extended = """ + Examples: + > SELECT _FUNC_(); + 0.9629742951434543 + > SELECT _FUNC_(0); + 0.8446490682263027 + """) +// scalastyle:on line.size.limit case class Rand(seed: Long) extends RDG { override protected def evalInternal(input: InternalRow): Double = rng.nextDouble() @@ -78,9 +87,18 @@ case class Rand(seed: Long) extends RDG { } } -/** Generate a random column with i.i.d. gaussian random distribution. */ +/** Generate a random column with i.i.d. values drawn from the standard normal distribution. */ +// scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(a) - Returns a random column with i.i.d. gaussian random distribution.") + usage = "_FUNC_([seed]) - Returns a random value with independent and identically distributed (i.i.d.) values drawn from the standard normal distribution.", + extended = """ + Examples: + > SELECT _FUNC_(); + -0.3254147983080288 + > SELECT _FUNC_(0); + 1.1164209726833079 + """) +// scalastyle:on line.size.limit case class Randn(seed: Long) extends RDG { override protected def evalInternal(input: InternalRow): Double = rng.nextGaussian() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala index d25da3fd587b6119469d1912cdc6988633f33578..5648ad6b6dc1837858b65559838decc579c64b2d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala @@ -68,7 +68,7 @@ trait StringRegexExpression extends ImplicitCastInputTypes { * Simple RegEx pattern matching function */ @ExpressionDescription( - usage = "str _FUNC_ pattern - Returns true if str matches pattern and false otherwise.") + usage = "str _FUNC_ pattern - Returns true if `str` matches `pattern`, or false otherwise.") case class Like(left: Expression, right: Expression) extends BinaryExpression with StringRegexExpression { @@ -121,7 +121,7 @@ case class Like(left: Expression, right: Expression) } @ExpressionDescription( - usage = "str _FUNC_ regexp - Returns true if str matches regexp and false otherwise.") + usage = "str _FUNC_ regexp - Returns true if `str` matches `regexp`, or false otherwise.") case class RLike(left: Expression, right: Expression) extends BinaryExpression with StringRegexExpression { @@ -175,8 +175,12 @@ case class RLike(left: Expression, right: Expression) * Splits str around pat (pattern is a regular expression). */ @ExpressionDescription( - usage = "_FUNC_(str, regex) - Splits str around occurrences that match regex", - extended = "> SELECT _FUNC_('oneAtwoBthreeC', '[ABC]');\n ['one', 'two', 'three']") + usage = "_FUNC_(str, regex) - Splits `str` around occurrences that match `regex`.", + extended = """ + Examples: + > SELECT _FUNC_('oneAtwoBthreeC', '[ABC]'); + ["one","two","three",""] + """) case class StringSplit(str: Expression, pattern: Expression) extends BinaryExpression with ImplicitCastInputTypes { @@ -206,9 +210,15 @@ case class StringSplit(str: Expression, pattern: Expression) * * NOTE: this expression is not THREAD-SAFE, as it has some internal mutable status. */ +// scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(str, regexp, rep) - replace all substrings of str that match regexp with rep.", - extended = "> SELECT _FUNC_('100-200', '(\\d+)', 'num');\n 'num-num'") + usage = "_FUNC_(str, regexp, rep) - Replaces all substrings of `str` that match `regexp` with `rep`.", + extended = """ + Examples: + > SELECT _FUNC_('100-200', '(\d+)', 'num'); + num-num + """) +// scalastyle:on line.size.limit case class RegExpReplace(subject: Expression, regexp: Expression, rep: Expression) extends TernaryExpression with ImplicitCastInputTypes { @@ -309,8 +319,12 @@ case class RegExpReplace(subject: Expression, regexp: Expression, rep: Expressio * NOTE: this expression is not THREAD-SAFE, as it has some internal mutable status. */ @ExpressionDescription( - usage = "_FUNC_(str, regexp[, idx]) - extracts a group that matches regexp.", - extended = "> SELECT _FUNC_('100-200', '(\\d+)-(\\d+)', 1);\n '100'") + usage = "_FUNC_(str, regexp[, idx]) - Extracts a group that matches `regexp`.", + extended = """ + Examples: + > SELECT _FUNC_('100-200', '(\d+)-(\d+)', 1); + 100 + """) case class RegExpExtract(subject: Expression, regexp: Expression, idx: Expression) extends TernaryExpression with ImplicitCastInputTypes { def this(s: Expression, r: Expression) = this(s, r, Literal(1)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala index 25a5e3fd7da733629e1a2d03cbe32b37236cece2..5f533fecf8d07ea6a7305e3de880b99efeb762cb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala @@ -40,9 +40,15 @@ import org.apache.spark.unsafe.types.{ByteArray, UTF8String} * An expression that concatenates multiple input strings into a single string. * If any input is null, concat returns null. */ +// scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(str1, str2, ..., strN) - Returns the concatenation of str1, str2, ..., strN", - extended = "> SELECT _FUNC_('Spark','SQL');\n 'SparkSQL'") + usage = "_FUNC_(str1, str2, ..., strN) - Returns the concatenation of `str1`, `str2`, ..., `strN`.", + extended = """ + Examples: + > SELECT _FUNC_('Spark','SQL'); + SparkSQL + """) +// scalastyle:on line.size.limit case class Concat(children: Seq[Expression]) extends Expression with ImplicitCastInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq.fill(children.size)(StringType) @@ -78,10 +84,15 @@ case class Concat(children: Seq[Expression]) extends Expression with ImplicitCas * * Returns null if the separator is null. Otherwise, concat_ws skips all null values. */ +// scalastyle:off line.size.limit @ExpressionDescription( - usage = - "_FUNC_(sep, [str | array(str)]+) - Returns the concatenation of the strings separated by sep.", - extended = "> SELECT _FUNC_(' ', Spark', 'SQL');\n 'Spark SQL'") + usage = "_FUNC_(sep, [str | array(str)]+) - Returns the concatenation of the strings separated by `sep`.", + extended = """ + Examples: + > SELECT _FUNC_(' ', Spark', 'SQL'); + Spark SQL + """) +// scalastyle:on line.size.limit case class ConcatWs(children: Seq[Expression]) extends Expression with ImplicitCastInputTypes { @@ -167,9 +178,15 @@ case class ConcatWs(children: Seq[Expression]) } } +// scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(n, str1, str2, ...) - returns the n-th string, e.g. returns str2 when n is 2", - extended = "> SELECT _FUNC_(1, 'scala', 'java') FROM src LIMIT 1;\n" + "'scala'") + usage = "_FUNC_(n, str1, str2, ...) - Returns the `n`-th string, e.g., returns `str2` when `n` is 2.", + extended = """ + Examples: + > SELECT _FUNC_(1, 'scala', 'java'); + scala + """) +// scalastyle:on line.size.limit case class Elt(children: Seq[Expression]) extends Expression with ImplicitCastInputTypes { @@ -246,8 +263,12 @@ trait String2StringExpression extends ImplicitCastInputTypes { * A function that converts the characters of a string to uppercase. */ @ExpressionDescription( - usage = "_FUNC_(str) - Returns str with all characters changed to uppercase", - extended = "> SELECT _FUNC_('SparkSql');\n 'SPARKSQL'") + usage = "_FUNC_(str) - Returns `str` with all characters changed to uppercase.", + extended = """ + Examples: + > SELECT _FUNC_('SparkSql'); + SPARKSQL + """) case class Upper(child: Expression) extends UnaryExpression with String2StringExpression { @@ -262,8 +283,12 @@ case class Upper(child: Expression) * A function that converts the characters of a string to lowercase. */ @ExpressionDescription( - usage = "_FUNC_(str) - Returns str with all characters changed to lowercase", - extended = "> SELECT _FUNC_('SparkSql');\n 'sparksql'") + usage = "_FUNC_(str) - Returns `str` with all characters changed to lowercase.", + extended = """ + Examples: + > SELECT _FUNC_('SparkSql'); + sparksql + """) case class Lower(child: Expression) extends UnaryExpression with String2StringExpression { override def convert(v: UTF8String): UTF8String = v.toLowerCase @@ -347,8 +372,12 @@ object StringTranslate { */ // scalastyle:off line.size.limit @ExpressionDescription( - usage = """_FUNC_(input, from, to) - Translates the input string by replacing the characters present in the from string with the corresponding characters in the to string""", - extended = "> SELECT _FUNC_('AaBbCc', 'abc', '123');\n 'A1B2C3'") + usage = "_FUNC_(input, from, to) - Translates the `input` string by replacing the characters present in the `from` string with the corresponding characters in the `to` string.", + extended = """ + Examples: + > SELECT _FUNC_('AaBbCc', 'abc', '123'); + A1B2C3 + """) // scalastyle:on line.size.limit case class StringTranslate(srcExpr: Expression, matchingExpr: Expression, replaceExpr: Expression) extends TernaryExpression with ImplicitCastInputTypes { @@ -407,9 +436,15 @@ case class StringTranslate(srcExpr: Expression, matchingExpr: Expression, replac */ // scalastyle:off line.size.limit @ExpressionDescription( - usage = """_FUNC_(str, str_array) - Returns the index (1-based) of the given string (left) in the comma-delimited list (right). - Returns 0, if the string wasn't found or if the given string (left) contains a comma.""", - extended = "> SELECT _FUNC_('ab','abc,b,ab,c,def');\n 3") + usage = """ + _FUNC_(str, str_array) - Returns the index (1-based) of the given string (`str`) in the comma-delimited list (`str_array`). + Returns 0, if the string was not found or if the given string (`str`) contains a comma. + """, + extended = """ + Examples: + > SELECT _FUNC_('ab','abc,b,ab,c,def'); + 3 + """) // scalastyle:on case class FindInSet(left: Expression, right: Expression) extends BinaryExpression with ImplicitCastInputTypes { @@ -434,8 +469,12 @@ case class FindInSet(left: Expression, right: Expression) extends BinaryExpressi * A function that trim the spaces from both ends for the specified string. */ @ExpressionDescription( - usage = "_FUNC_(str) - Removes the leading and trailing space characters from str.", - extended = "> SELECT _FUNC_(' SparkSQL ');\n 'SparkSQL'") + usage = "_FUNC_(str) - Removes the leading and trailing space characters from `str`.", + extended = """ + Examples: + > SELECT _FUNC_(' SparkSQL '); + SparkSQL + """) case class StringTrim(child: Expression) extends UnaryExpression with String2StringExpression { @@ -452,8 +491,12 @@ case class StringTrim(child: Expression) * A function that trim the spaces from left end for given string. */ @ExpressionDescription( - usage = "_FUNC_(str) - Removes the leading space characters from str.", - extended = "> SELECT _FUNC_(' SparkSQL ');\n 'SparkSQL '") + usage = "_FUNC_(str) - Removes the leading and trailing space characters from `str`.", + extended = """ + Examples: + > SELECT _FUNC_(' SparkSQL'); + SparkSQL + """) case class StringTrimLeft(child: Expression) extends UnaryExpression with String2StringExpression { @@ -470,8 +513,12 @@ case class StringTrimLeft(child: Expression) * A function that trim the spaces from right end for given string. */ @ExpressionDescription( - usage = "_FUNC_(str) - Removes the trailing space characters from str.", - extended = "> SELECT _FUNC_(' SparkSQL ');\n ' SparkSQL'") + usage = "_FUNC_(str) - Removes the trailing space characters from `str`.", + extended = """ + Examples: + > SELECT _FUNC_(' SparkSQL '); + SparkSQL + """) case class StringTrimRight(child: Expression) extends UnaryExpression with String2StringExpression { @@ -492,8 +539,12 @@ case class StringTrimRight(child: Expression) * NOTE: that this is not zero based, but 1-based index. The first character in str has index 1. */ @ExpressionDescription( - usage = "_FUNC_(str, substr) - Returns the (1-based) index of the first occurrence of substr in str.", - extended = "> SELECT _FUNC_('SparkSQL', 'SQL');\n 6") + usage = "_FUNC_(str, substr) - Returns the (1-based) index of the first occurrence of `substr` in `str`.", + extended = """ + Examples: + > SELECT _FUNC_('SparkSQL', 'SQL'); + 6 + """) case class StringInstr(str: Expression, substr: Expression) extends BinaryExpression with ImplicitCastInputTypes { @@ -522,12 +573,18 @@ case class StringInstr(str: Expression, substr: Expression) */ // scalastyle:off line.size.limit @ExpressionDescription( - usage = """_FUNC_(str, delim, count) - Returns the substring from str before count occurrences of the delimiter delim. - If count is positive, everything to the left of the final delimiter (counting from the - left) is returned. If count is negative, everything to the right of the final delimiter - (counting from the right) is returned. Substring_index performs a case-sensitive match - when searching for delim.""", - extended = "> SELECT _FUNC_('www.apache.org', '.', 2);\n 'www.apache'") + usage = """ + _FUNC_(str, delim, count) - Returns the substring from `str` before `count` occurrences of the delimiter `delim`. + If `count` is positive, everything to the left of the final delimiter (counting from the + left) is returned. If `count` is negative, everything to the right of the final delimiter + (counting from the right) is returned. The function substring_index performs a case-sensitive match + when searching for `delim`. + """, + extended = """ + Examples: + > SELECT _FUNC_('www.apache.org', '.', 2); + www.apache + """) // scalastyle:on line.size.limit case class SubstringIndex(strExpr: Expression, delimExpr: Expression, countExpr: Expression) extends TernaryExpression with ImplicitCastInputTypes { @@ -554,9 +611,15 @@ case class SubstringIndex(strExpr: Expression, delimExpr: Expression, countExpr: */ // scalastyle:off line.size.limit @ExpressionDescription( - usage = """_FUNC_(substr, str[, pos]) - Returns the position of the first occurrence of substr in str after position pos. - The given pos and return value are 1-based.""", - extended = "> SELECT _FUNC_('bar', 'foobarbar', 5);\n 7") + usage = """ + _FUNC_(substr, str[, pos]) - Returns the position of the first occurrence of `substr` in `str` after position `pos`. + The given `pos` and return value are 1-based. + """, + extended = """ + Examples: + > SELECT _FUNC_('bar', 'foobarbar', 5); + 7 + """) // scalastyle:on line.size.limit case class StringLocate(substr: Expression, str: Expression, start: Expression) extends TernaryExpression with ImplicitCastInputTypes { @@ -631,10 +694,17 @@ case class StringLocate(substr: Expression, str: Expression, start: Expression) * Returns str, left-padded with pad to a length of len. */ @ExpressionDescription( - usage = """_FUNC_(str, len, pad) - Returns str, left-padded with pad to a length of len. - If str is longer than len, the return value is shortened to len characters.""", - extended = "> SELECT _FUNC_('hi', 5, '??');\n '???hi'\n" + - "> SELECT _FUNC_('hi', 1, '??');\n 'h'") + usage = """ + _FUNC_(str, len, pad) - Returns `str`, left-padded with `pad` to a length of `len`. + If `str` is longer than `len`, the return value is shortened to `len` characters. + """, + extended = """ + Examples: + > SELECT _FUNC_('hi', 5, '??'); + ???hi + > SELECT _FUNC_('hi', 1, '??'); + h + """) case class StringLPad(str: Expression, len: Expression, pad: Expression) extends TernaryExpression with ImplicitCastInputTypes { @@ -657,10 +727,17 @@ case class StringLPad(str: Expression, len: Expression, pad: Expression) * Returns str, right-padded with pad to a length of len. */ @ExpressionDescription( - usage = """_FUNC_(str, len, pad) - Returns str, right-padded with pad to a length of len. - If str is longer than len, the return value is shortened to len characters.""", - extended = "> SELECT _FUNC_('hi', 5, '??');\n 'hi???'\n" + - "> SELECT _FUNC_('hi', 1, '??');\n 'h'") + usage = """ + _FUNC_(str, len, pad) - Returns `str`, right-padded with `pad` to a length of `len`. + If `str` is longer than `len`, the return value is shortened to `len` characters. + """, + extended = """ + Examples: + > SELECT _FUNC_('hi', 5, '??'); + hi??? + > SELECT _FUNC_('hi', 1, '??'); + h + """) case class StringRPad(str: Expression, len: Expression, pad: Expression) extends TernaryExpression with ImplicitCastInputTypes { @@ -696,16 +773,16 @@ object ParseUrl { * Extracts a part from a URL */ @ExpressionDescription( - usage = "_FUNC_(url, partToExtract[, key]) - extracts a part from a URL", - extended = """Parts: HOST, PATH, QUERY, REF, PROTOCOL, AUTHORITY, FILE, USERINFO. - Key specifies which query to extract. + usage = "_FUNC_(url, partToExtract[, key]) - Extracts a part from a URL.", + extended = """ Examples: > SELECT _FUNC_('http://spark.apache.org/path?query=1', 'HOST') - 'spark.apache.org' + spark.apache.org > SELECT _FUNC_('http://spark.apache.org/path?query=1', 'QUERY') - 'query=1' + query=1 > SELECT _FUNC_('http://spark.apache.org/path?query=1', 'QUERY', 'query') - '1'""") + 1 + """) case class ParseUrl(children: Seq[Expression]) extends Expression with ExpectsInputTypes with CodegenFallback { @@ -851,8 +928,12 @@ case class ParseUrl(children: Seq[Expression]) */ // scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(String format, Obj... args) - Returns a formatted string from printf-style format strings.", - extended = "> SELECT _FUNC_(\"Hello World %d %s\", 100, \"days\");\n 'Hello World 100 days'") + usage = "_FUNC_(strfmt, obj, ...) - Returns a formatted string from printf-style format strings.", + extended = """ + Examples: + > SELECT _FUNC_("Hello World %d %s", 100, "days"); + Hello World 100 days + """) // scalastyle:on line.size.limit case class FormatString(children: Expression*) extends Expression with ImplicitCastInputTypes { @@ -923,10 +1004,15 @@ case class FormatString(children: Expression*) extends Expression with ImplicitC * Words are delimited by whitespace. */ @ExpressionDescription( - usage = - """_FUNC_(str) - Returns str with the first letter of each word in uppercase. - All other letters are in lowercase. Words are delimited by white space.""", - extended = "> SELECT initcap('sPark sql');\n 'Spark Sql'") + usage = """ + _FUNC_(str) - Returns `str` with the first letter of each word in uppercase. + All other letters are in lowercase. Words are delimited by white space. + """, + extended = """ + Examples: + > SELECT initcap('sPark sql'); + Spark Sql + """) case class InitCap(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def inputTypes: Seq[DataType] = Seq(StringType) @@ -944,8 +1030,12 @@ case class InitCap(child: Expression) extends UnaryExpression with ImplicitCastI * Returns the string which repeat the given string value n times. */ @ExpressionDescription( - usage = "_FUNC_(str, n) - Returns the string which repeat the given string value n times.", - extended = "> SELECT _FUNC_('123', 2);\n '123123'") + usage = "_FUNC_(str, n) - Returns the string which repeats the given string value n times.", + extended = """ + Examples: + > SELECT _FUNC_('123', 2); + 123123 + """) case class StringRepeat(str: Expression, times: Expression) extends BinaryExpression with ImplicitCastInputTypes { @@ -970,7 +1060,11 @@ case class StringRepeat(str: Expression, times: Expression) */ @ExpressionDescription( usage = "_FUNC_(str) - Returns the reversed given string.", - extended = "> SELECT _FUNC_('Spark SQL');\n 'LQS krapS'") + extended = """ + Examples: + > SELECT _FUNC_('Spark SQL'); + LQS krapS + """) case class StringReverse(child: Expression) extends UnaryExpression with String2StringExpression { override def convert(v: UTF8String): UTF8String = v.reverse() @@ -982,11 +1076,15 @@ case class StringReverse(child: Expression) extends UnaryExpression with String2 } /** - * Returns a n spaces string. + * Returns a string consisting of n spaces. */ @ExpressionDescription( - usage = "_FUNC_(n) - Returns a n spaces string.", - extended = "> SELECT _FUNC_(2);\n ' '") + usage = "_FUNC_(n) - Returns a string consisting of `n` spaces.", + extended = """ + Examples: + > SELECT concat(_FUNC_(2), '1'); + 1 + """) case class StringSpace(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { @@ -1014,8 +1112,16 @@ case class StringSpace(child: Expression) */ // scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(str, pos[, len]) - Returns the substring of str that starts at pos and is of length len or the slice of byte array that starts at pos and is of length len.", - extended = "> SELECT _FUNC_('Spark SQL', 5);\n 'k SQL'\n> SELECT _FUNC_('Spark SQL', -3);\n 'SQL'\n> SELECT _FUNC_('Spark SQL', 5, 1);\n 'k'") + usage = "_FUNC_(str, pos[, len]) - Returns the substring of `str` that starts at `pos` and is of length `len`, or the slice of byte array that starts at `pos` and is of length `len`.", + extended = """ + Examples: + > SELECT _FUNC_('Spark SQL', 5); + k SQL + > SELECT _FUNC_('Spark SQL', -3); + SQL + > SELECT _FUNC_('Spark SQL', 5, 1); + k + """) // scalastyle:on line.size.limit case class Substring(str: Expression, pos: Expression, len: Expression) extends TernaryExpression with ImplicitCastInputTypes { @@ -1055,8 +1161,12 @@ case class Substring(str: Expression, pos: Expression, len: Expression) * A function that return the length of the given string or binary expression. */ @ExpressionDescription( - usage = "_FUNC_(str | binary) - Returns the length of str or number of bytes in binary data.", - extended = "> SELECT _FUNC_('Spark SQL');\n 9") + usage = "_FUNC_(expr) - Returns the length of `expr` or number of bytes in binary data.", + extended = """ + Examples: + > SELECT _FUNC_('Spark SQL'); + 9 + """) case class Length(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def dataType: DataType = IntegerType override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(StringType, BinaryType)) @@ -1079,7 +1189,11 @@ case class Length(child: Expression) extends UnaryExpression with ImplicitCastIn */ @ExpressionDescription( usage = "_FUNC_(str1, str2) - Returns the Levenshtein distance between the two given strings.", - extended = "> SELECT _FUNC_('kitten', 'sitting');\n 3") + extended = """ + Examples: + > SELECT _FUNC_('kitten', 'sitting'); + 3 + """) case class Levenshtein(left: Expression, right: Expression) extends BinaryExpression with ImplicitCastInputTypes { @@ -1096,11 +1210,15 @@ case class Levenshtein(left: Expression, right: Expression) extends BinaryExpres } /** - * A function that return soundex code of the given string expression. + * A function that return Soundex code of the given string expression. */ @ExpressionDescription( - usage = "_FUNC_(str) - Returns soundex code of the string.", - extended = "> SELECT _FUNC_('Miller');\n 'M460'") + usage = "_FUNC_(str) - Returns Soundex code of the string.", + extended = """ + Examples: + > SELECT _FUNC_('Miller'); + M460 + """) case class SoundEx(child: Expression) extends UnaryExpression with ExpectsInputTypes { override def dataType: DataType = StringType @@ -1118,9 +1236,14 @@ case class SoundEx(child: Expression) extends UnaryExpression with ExpectsInputT * Returns the numeric value of the first character of str. */ @ExpressionDescription( - usage = "_FUNC_(str) - Returns the numeric value of the first character of str.", - extended = "> SELECT _FUNC_('222');\n 50\n" + - "> SELECT _FUNC_(2);\n 50") + usage = "_FUNC_(str) - Returns the numeric value of the first character of `str`.", + extended = """ + Examples: + > SELECT _FUNC_('222'); + 50 + > SELECT _FUNC_(2); + 50 + """) case class Ascii(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def dataType: DataType = IntegerType @@ -1153,7 +1276,12 @@ case class Ascii(child: Expression) extends UnaryExpression with ImplicitCastInp * Converts the argument from binary to a base 64 string. */ @ExpressionDescription( - usage = "_FUNC_(bin) - Convert the argument from binary to a base 64 string.") + usage = "_FUNC_(bin) - Converts the argument from a binary `bin` to a base 64 string.", + extended = """ + Examples: + > SELECT _FUNC_('Spark SQL'); + U3BhcmsgU1FM + """) case class Base64(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def dataType: DataType = StringType @@ -1177,7 +1305,12 @@ case class Base64(child: Expression) extends UnaryExpression with ImplicitCastIn * Converts the argument from a base 64 string to BINARY. */ @ExpressionDescription( - usage = "_FUNC_(str) - Convert the argument from a base 64 string to binary.") + usage = "_FUNC_(str) - Converts the argument from a base 64 string `str` to a binary.", + extended = """ + Examples: + > SELECT _FUNC_('U3BhcmsgU1FM'); + Spark SQL + """) case class UnBase64(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def dataType: DataType = BinaryType @@ -1199,8 +1332,15 @@ case class UnBase64(child: Expression) extends UnaryExpression with ImplicitCast * (one of 'US-ASCII', 'ISO-8859-1', 'UTF-8', 'UTF-16BE', 'UTF-16LE', 'UTF-16'). * If either argument is null, the result will also be null. */ +// scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(bin, str) - Decode the first argument using the second argument character set.") + usage = "_FUNC_(bin, charset) - Decodes the first argument using the second argument character set.", + extended = """ + Examples: + > SELECT _FUNC_(encode('abc', 'utf-8'), 'utf-8'); + abc + """) +// scalastyle:on line.size.limit case class Decode(bin: Expression, charset: Expression) extends BinaryExpression with ImplicitCastInputTypes { @@ -1231,8 +1371,15 @@ case class Decode(bin: Expression, charset: Expression) * (one of 'US-ASCII', 'ISO-8859-1', 'UTF-8', 'UTF-16BE', 'UTF-16LE', 'UTF-16'). * If either argument is null, the result will also be null. */ +// scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(str, str) - Encode the first argument using the second argument character set.") + usage = "_FUNC_(str, charset) - Encodes the first argument using the second argument character set.", + extended = """ + Examples: + > SELECT _FUNC_('abc', 'utf-8'); + abc + """) +// scalastyle:on line.size.limit case class Encode(value: Expression, charset: Expression) extends BinaryExpression with ImplicitCastInputTypes { @@ -1263,10 +1410,16 @@ case class Encode(value: Expression, charset: Expression) * fractional part. */ @ExpressionDescription( - usage = """_FUNC_(X, D) - Formats the number X like '#,###,###.##', rounded to D decimal places. - If D is 0, the result has no decimal point or fractional part. - This is supposed to function like MySQL's FORMAT.""", - extended = "> SELECT _FUNC_(12332.123456, 4);\n '12,332.1235'") + usage = """ + _FUNC_(expr1, expr2) - Formats the number `expr1` like '#,###,###.##', rounded to `expr2` + decimal places. If `expr2` is 0, the result has no decimal point or fractional part. + This is supposed to function like MySQL's FORMAT. + """, + extended = """ + Examples: + > SELECT _FUNC_(12332.123456, 4); + 12,332.1235 + """) case class FormatNumber(x: Expression, d: Expression) extends BinaryExpression with ExpectsInputTypes { @@ -1388,8 +1541,12 @@ case class FormatNumber(x: Expression, d: Expression) * The 'lang' and 'country' arguments are optional, and if omitted, the default locale is used. */ @ExpressionDescription( - usage = "_FUNC_(str[, lang, country]) - Splits str into an array of array of words.", - extended = "> SELECT _FUNC_('Hi there! Good morning.');\n [['Hi','there'], ['Good','morning']]") + usage = "_FUNC_(str[, lang, country]) - Splits `str` into an array of array of words.", + extended = """ + Examples: + > SELECT _FUNC_('Hi there! Good morning.'); + [["Hi","there"],["Good","morning"]] + """) case class Sentences( str: Expression, language: Expression = Literal(""), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala index b47486f7af7f9e9b356582754efe64832ef73208..3cbbcdf4a96cc7d5f6d60aec903e8610c21e1c1f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala @@ -321,7 +321,7 @@ abstract class OffsetWindowFunction val input: Expression /** - * Default result value for the function when the 'offset'th row does not exist. + * Default result value for the function when the `offset`th row does not exist. */ val default: Expression @@ -372,22 +372,23 @@ abstract class OffsetWindowFunction } /** - * The Lead function returns the value of 'x' at the 'offset'th row after the current row in + * The Lead function returns the value of `input` at the `offset`th row after the current row in * the window. Offsets start at 0, which is the current row. The offset must be constant - * integer value. The default offset is 1. When the value of 'x' is null at the 'offset'th row, - * null is returned. If there is no such offset row, the default expression is evaluated. + * integer value. The default offset is 1. When the value of `input` is null at the `offset`th row, + * null is returned. If there is no such offset row, the `default` expression is evaluated. * - * @param input expression to evaluate 'offset' rows after the current row. + * @param input expression to evaluate `offset` rows after the current row. * @param offset rows to jump ahead in the partition. * @param default to use when the offset is larger than the window. The default value is null. */ -@ExpressionDescription(usage = - """_FUNC_(input, offset, default) - LEAD returns the value of 'x' at the 'offset'th row - after the current row in the window. - The default value of 'offset' is 1 and the default value of 'default' is null. - If the value of 'x' at the 'offset'th row is null, null is returned. - If there is no such offset row (e.g. when the offset is 1, the last row of the window - does not have any subsequent row), 'default' is returned.""") +@ExpressionDescription( + usage = """ + _FUNC_(input[, offset[, default]]) - Returns the value of `input` at the `offset`th row + after the current row in the window. The default value of `offset` is 1 and the default + value of `default` is null. If the value of `input` at the `offset`th row is null, + null is returned. If there is no such an offset row (e.g., when the offset is 1, the last + row of the window does not have any subsequent row), `default` is returned. + """) case class Lead(input: Expression, offset: Expression, default: Expression) extends OffsetWindowFunction { @@ -401,22 +402,23 @@ case class Lead(input: Expression, offset: Expression, default: Expression) } /** - * The Lag function returns the value of 'x' at the 'offset'th row before the current row in + * The Lag function returns the value of `input` at the `offset`th row before the current row in * the window. Offsets start at 0, which is the current row. The offset must be constant - * integer value. The default offset is 1. When the value of 'x' is null at the 'offset'th row, - * null is returned. If there is no such offset row, the default expression is evaluated. + * integer value. The default offset is 1. When the value of `input` is null at the `offset`th row, + * null is returned. If there is no such offset row, the `default` expression is evaluated. * - * @param input expression to evaluate 'offset' rows before the current row. + * @param input expression to evaluate `offset` rows before the current row. * @param offset rows to jump back in the partition. * @param default to use when the offset row does not exist. */ -@ExpressionDescription(usage = - """_FUNC_(input, offset, default) - LAG returns the value of 'x' at the 'offset'th row - before the current row in the window. - The default value of 'offset' is 1 and the default value of 'default' is null. - If the value of 'x' at the 'offset'th row is null, null is returned. - If there is no such offset row (e.g. when the offset is 1, the first row of the window - does not have any previous row), 'default' is returned.""") +@ExpressionDescription( + usage = """ + _FUNC_(input[, offset[, default]]) - Returns the value of `input` at the `offset`th row + before the current row in the window. The default value of `offset` is 1 and the default + value of `default` is null. If the value of `input` at the `offset`th row is null, + null is returned. If there is no such offset row (e.g., when the offset is 1, the first + row of the window does not have any previous row), `default` is returned. + """) case class Lag(input: Expression, offset: Expression, default: Expression) extends OffsetWindowFunction { @@ -471,26 +473,28 @@ object SizeBasedWindowFunction { * * This documentation has been based upon similar documentation for the Hive and Presto projects. */ -@ExpressionDescription(usage = - """_FUNC_() - The ROW_NUMBER() function assigns a unique, sequential number to - each row, starting with one, according to the ordering of rows within - the window partition.""") +@ExpressionDescription( + usage = """ + _FUNC_() - Assigns a unique, sequential number to each row, starting with one, + according to the ordering of rows within the window partition. + """) case class RowNumber() extends RowNumberLike { override val evaluateExpression = rowNumber override def prettyName: String = "row_number" } /** - * The CumeDist function computes the position of a value relative to a all values in the partition. + * The CumeDist function computes the position of a value relative to all values in the partition. * The result is the number of rows preceding or equal to the current row in the ordering of the * partition divided by the total number of rows in the window partition. Any tie values in the * ordering will evaluate to the same position. * * This documentation has been based upon similar documentation for the Hive and Presto projects. */ -@ExpressionDescription(usage = - """_FUNC_() - The CUME_DIST() function computes the position of a value relative to - a all values in the partition.""") +@ExpressionDescription( + usage = """ + _FUNC_() - Computes the position of a value relative to all values in the partition. + """) case class CumeDist() extends RowNumberLike with SizeBasedWindowFunction { override def dataType: DataType = DoubleType // The frame for CUME_DIST is Range based instead of Row based, because CUME_DIST must @@ -501,8 +505,8 @@ case class CumeDist() extends RowNumberLike with SizeBasedWindowFunction { } /** - * The NTile function divides the rows for each window partition into 'n' buckets ranging from 1 to - * at most 'n'. Bucket values will differ by at most 1. If the number of rows in the partition does + * The NTile function divides the rows for each window partition into `n` buckets ranging from 1 to + * at most `n`. Bucket values will differ by at most 1. If the number of rows in the partition does * not divide evenly into the number of buckets, then the remainder values are distributed one per * bucket, starting with the first bucket. * @@ -521,9 +525,11 @@ case class CumeDist() extends RowNumberLike with SizeBasedWindowFunction { * * @param buckets number of buckets to divide the rows in. Default value is 1. */ -@ExpressionDescription(usage = - """_FUNC_(x) - The NTILE(n) function divides the rows for each window partition - into 'n' buckets ranging from 1 to at most 'n'.""") +@ExpressionDescription( + usage = """ + _FUNC_(n) - Divides the rows for each window partition into `n` buckets ranging + from 1 to at most `n`. + """) case class NTile(buckets: Expression) extends RowNumberLike with SizeBasedWindowFunction { def this() = this(Literal(1)) @@ -587,9 +593,9 @@ case class NTile(buckets: Expression) extends RowNumberLike with SizeBasedWindow /** * A RankLike function is a WindowFunction that changes its value based on a change in the value of - * the order of the window in which is processed. For instance, when the value of 'x' changes in a - * window ordered by 'x' the rank function also changes. The size of the change of the rank function - * is (typically) not dependent on the size of the change in 'x'. + * the order of the window in which is processed. For instance, when the value of `input` changes + * in a window ordered by `input` the rank function also changes. The size of the change of the + * rank function is (typically) not dependent on the size of the change in `input`. * * This documentation has been based upon similar documentation for the Hive and Presto projects. */ @@ -635,7 +641,7 @@ abstract class RankLike extends AggregateWindowFunction { /** * The Rank function computes the rank of a value in a group of values. The result is one plus the - * number of rows preceding or equal to the current row in the ordering of the partition. Tie values + * number of rows preceding or equal to the current row in the ordering of the partition. The values * will produce gaps in the sequence. * * This documentation has been based upon similar documentation for the Hive and Presto projects. @@ -644,10 +650,12 @@ abstract class RankLike extends AggregateWindowFunction { * change in rank. This is an internal parameter and will be assigned by the * Analyser. */ -@ExpressionDescription(usage = - """_FUNC_() - RANK() computes the rank of a value in a group of values. The result - is one plus the number of rows preceding or equal to the current row in the - ordering of the partition. Tie values will produce gaps in the sequence.""") +@ExpressionDescription( + usage = """ + _FUNC_() - Computes the rank of a value in a group of values. The result is one plus the number + of rows preceding or equal to the current row in the ordering of the partition. The values + will produce gaps in the sequence. + """) case class Rank(children: Seq[Expression]) extends RankLike { def this() = this(Nil) override def withOrder(order: Seq[Expression]): Rank = Rank(order) @@ -655,8 +663,8 @@ case class Rank(children: Seq[Expression]) extends RankLike { /** * The DenseRank function computes the rank of a value in a group of values. The result is one plus - * the previously assigned rank value. Unlike Rank, DenseRank will not produce gaps in the ranking - * sequence. + * the previously assigned rank value. Unlike [[Rank]], [[DenseRank]] will not produce gaps in the + * ranking sequence. * * This documentation has been based upon similar documentation for the Hive and Presto projects. * @@ -664,10 +672,12 @@ case class Rank(children: Seq[Expression]) extends RankLike { * change in rank. This is an internal parameter and will be assigned by the * Analyser. */ -@ExpressionDescription(usage = - """_FUNC_() - The DENSE_RANK() function computes the rank of a value in a group of - values. The result is one plus the previously assigned rank value. Unlike Rank, - DenseRank will not produce gaps in the ranking sequence.""") +@ExpressionDescription( + usage = """ + _FUNC_() - Computes the rank of a value in a group of values. The result is one plus the + previously assigned rank value. Unlike the function rank, dense_rank will not produce gaps + in the ranking sequence. + """) case class DenseRank(children: Seq[Expression]) extends RankLike { def this() = this(Nil) override def withOrder(order: Seq[Expression]): DenseRank = DenseRank(order) @@ -692,9 +702,10 @@ case class DenseRank(children: Seq[Expression]) extends RankLike { * change in rank. This is an internal parameter and will be assigned by the * Analyser. */ -@ExpressionDescription(usage = - """_FUNC_() - PERCENT_RANK() The PercentRank function computes the percentage - ranking of a value in a group of values.""") +@ExpressionDescription( + usage = """ + _FUNC_() - Computes the percentage ranking of a value in a group of values. + """) case class PercentRank(children: Seq[Expression]) extends RankLike with SizeBasedWindowFunction { def this() = this(Nil) override def withOrder(order: Seq[Expression]): PercentRank = PercentRank(order) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/xpath.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/xpath.scala index 47f039e6a4cc490fc01cdd90df243fb7e2a49289..aa328045cafdb1a7122547e86a889d143b0ebea7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/xpath.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/xpath.scala @@ -55,9 +55,15 @@ abstract class XPathExtract extends BinaryExpression with ExpectsInputTypes with def path: Expression } +// scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(xml, xpath) - Evaluates a boolean xpath expression.", - extended = "> SELECT _FUNC_('<a><b>1</b></a>','a/b');\ntrue") + usage = "_FUNC_(xml, xpath) - Returns true if the XPath expression evaluates to true, or if a matching node is found.", + extended = """ + Examples: + > SELECT _FUNC_('<a><b>1</b></a>','a/b'); + true + """) +// scalastyle:on line.size.limit case class XPathBoolean(xml: Expression, path: Expression) extends XPathExtract { override def prettyName: String = "xpath_boolean" @@ -68,11 +74,17 @@ case class XPathBoolean(xml: Expression, path: Expression) extends XPathExtract } } +// scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(xml, xpath) - Returns a short value that matches the xpath expression", - extended = "> SELECT _FUNC_('<a><b>1</b><b>2</b></a>','sum(a/b)');\n3") + usage = "_FUNC_(xml, xpath) - Returns a short integer value, or the value zero if no match is found, or a match is found but the value is non-numeric.", + extended = """ + Examples: + > SELECT _FUNC_('<a><b>1</b><b>2</b></a>', 'sum(a/b)'); + 3 + """) +// scalastyle:on line.size.limit case class XPathShort(xml: Expression, path: Expression) extends XPathExtract { - override def prettyName: String = "xpath_int" + override def prettyName: String = "xpath_short" override def dataType: DataType = ShortType override def nullSafeEval(xml: Any, path: Any): Any = { @@ -81,9 +93,15 @@ case class XPathShort(xml: Expression, path: Expression) extends XPathExtract { } } +// scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(xml, xpath) - Returns an integer value that matches the xpath expression", - extended = "> SELECT _FUNC_('<a><b>1</b><b>2</b></a>','sum(a/b)');\n3") + usage = "_FUNC_(xml, xpath) - Returns an integer value, or the value zero if no match is found, or a match is found but the value is non-numeric.", + extended = """ + Examples: + > SELECT _FUNC_('<a><b>1</b><b>2</b></a>', 'sum(a/b)'); + 3 + """) +// scalastyle:on line.size.limit case class XPathInt(xml: Expression, path: Expression) extends XPathExtract { override def prettyName: String = "xpath_int" override def dataType: DataType = IntegerType @@ -94,9 +112,15 @@ case class XPathInt(xml: Expression, path: Expression) extends XPathExtract { } } +// scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(xml, xpath) - Returns a long value that matches the xpath expression", - extended = "> SELECT _FUNC_('<a><b>1</b><b>2</b></a>','sum(a/b)');\n3") + usage = "_FUNC_(xml, xpath) - Returns a long integer value, or the value zero if no match is found, or a match is found but the value is non-numeric.", + extended = """ + Examples: + > SELECT _FUNC_('<a><b>1</b><b>2</b></a>', 'sum(a/b)'); + 3 + """) +// scalastyle:on line.size.limit case class XPathLong(xml: Expression, path: Expression) extends XPathExtract { override def prettyName: String = "xpath_long" override def dataType: DataType = LongType @@ -107,9 +131,15 @@ case class XPathLong(xml: Expression, path: Expression) extends XPathExtract { } } +// scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(xml, xpath) - Returns a float value that matches the xpath expression", - extended = "> SELECT _FUNC_('<a><b>1</b><b>2</b></a>','sum(a/b)');\n3.0") + usage = "_FUNC_(xml, xpath) - Returns a float value, the value zero if no match is found, or NaN if a match is found but the value is non-numeric.", + extended = """ + Examples: + > SELECT _FUNC_('<a><b>1</b><b>2</b></a>', 'sum(a/b)'); + 3.0 + """) +// scalastyle:on line.size.limit case class XPathFloat(xml: Expression, path: Expression) extends XPathExtract { override def prettyName: String = "xpath_float" override def dataType: DataType = FloatType @@ -120,9 +150,15 @@ case class XPathFloat(xml: Expression, path: Expression) extends XPathExtract { } } +// scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(xml, xpath) - Returns a double value that matches the xpath expression", - extended = "> SELECT _FUNC_('<a><b>1</b><b>2</b></a>','sum(a/b)');\n3.0") + usage = "_FUNC_(xml, xpath) - Returns a double value, the value zero if no match is found, or NaN if a match is found but the value is non-numeric.", + extended = """ + Examples: + > SELECT _FUNC_('<a><b>1</b><b>2</b></a>', 'sum(a/b)'); + 3.0 + """) +// scalastyle:on line.size.limit case class XPathDouble(xml: Expression, path: Expression) extends XPathExtract { override def prettyName: String = "xpath_float" override def dataType: DataType = DoubleType @@ -135,8 +171,12 @@ case class XPathDouble(xml: Expression, path: Expression) extends XPathExtract { // scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(xml, xpath) - Returns the text contents of the first xml node that matches the xpath expression", - extended = "> SELECT _FUNC_('<a><b>b</b><c>cc</c></a>','a/c');\ncc") + usage = "_FUNC_(xml, xpath) - Returns the text contents of the first xml node that matches the XPath expression.", + extended = """ + Examples: + > SELECT _FUNC_('<a><b>b</b><c>cc</c></a>','a/c'); + cc + """) // scalastyle:on line.size.limit case class XPathString(xml: Expression, path: Expression) extends XPathExtract { override def prettyName: String = "xpath_string" @@ -150,8 +190,12 @@ case class XPathString(xml: Expression, path: Expression) extends XPathExtract { // scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(xml, xpath) - Returns a string array of values within xml nodes that match the xpath expression", - extended = "> SELECT _FUNC_('<a><b>b1</b><b>b2</b><b>b3</b><c>c1</c><c>c2</c></a>','a/b/text()');\n['b1','b2','b3']") + usage = "_FUNC_(xml, xpath) - Returns a string array of values within the nodes of xml that match the XPath expression.", + extended = """ + Examples: + > SELECT _FUNC_('<a><b>b1</b><b>b2</b><b>b3</b><c>c1</c><c>c2</c></a>','a/b/text()'); + ['b1','b2','b3'] + """) // scalastyle:on line.size.limit case class XPathList(xml: Expression, path: Expression) extends XPathExtract { override def prettyName: String = "xpath" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala index 24d825f5cb33a6f0b2060191eb3b643acd0c0f00..ea5398761c46de80f82dc1c54aa14fd3ccbb293f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala @@ -103,18 +103,22 @@ case class DescribeFunctionCommand( functionName.funcName.toLowerCase match { case "<>" => Row(s"Function: $functionName") :: - Row(s"Usage: a <> b - Returns TRUE if a is not equal to b") :: Nil + Row("Usage: expr1 <> expr2 - " + + "Returns true if `expr1` is not equal to `expr2`.") :: Nil case "!=" => Row(s"Function: $functionName") :: - Row(s"Usage: a != b - Returns TRUE if a is not equal to b") :: Nil + Row("Usage: expr1 != expr2 - " + + "Returns true if `expr1` is not equal to `expr2`.") :: Nil case "between" => - Row(s"Function: between") :: - Row(s"Usage: a [NOT] BETWEEN b AND c - " + - s"evaluate if a is [not] in between b and c") :: Nil + Row("Function: between") :: + Row("Usage: expr1 [NOT] BETWEEN expr2 AND expr3 - " + + "evaluate if `expr1` is [not] in between `expr2` and `expr3`.") :: Nil case "case" => - Row(s"Function: case") :: - Row(s"Usage: CASE a WHEN b THEN c [WHEN d THEN e]* [ELSE f] END - " + - s"When a = b, returns c; when a = d, return e; else return f") :: Nil + Row("Function: case") :: + Row("Usage: CASE expr1 WHEN expr2 THEN expr3 " + + "[WHEN expr4 THEN expr5]* [ELSE expr6] END - " + + "When `expr1` = `expr2`, returns `expr3`; " + + "when `expr1` = `expr4`, return `expr5`; else return `expr6`.") :: Nil case _ => try { val info = sparkSession.sessionState.catalog.lookupFunctionInfo(functionName) @@ -126,7 +130,7 @@ case class DescribeFunctionCommand( if (isExtended) { result :+ - Row(s"Extended Usage:\n${replaceFunctionName(info.getExtended, name)}") + Row(s"Extended Usage:${replaceFunctionName(info.getExtended, info.getName)}") } else { result } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 9a3d93cf17b78af98c611011b86504c6f273e5a1..6b517bc70f7d21e3cf624b2aea4b7f5a30c25c3c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -85,15 +85,16 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { checkKeywordsExist(sql("describe function extended upper"), "Function: upper", "Class: org.apache.spark.sql.catalyst.expressions.Upper", - "Usage: upper(str) - Returns str with all characters changed to uppercase", + "Usage: upper(str) - Returns `str` with all characters changed to uppercase", "Extended Usage:", + "Examples:", "> SELECT upper('SparkSql');", - "'SPARKSQL'") + "SPARKSQL") checkKeywordsExist(sql("describe functioN Upper"), "Function: upper", "Class: org.apache.spark.sql.catalyst.expressions.Upper", - "Usage: upper(str) - Returns str with all characters changed to uppercase") + "Usage: upper(str) - Returns `str` with all characters changed to uppercase") checkKeywordsNotExist(sql("describe functioN Upper"), "Extended Usage") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index bde3c8a42e1c01cccdea53cc455fcb3433587ece..22d4c929bf565146a3f8bd4a1c7c33d11935f6b0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -1445,34 +1445,34 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { sql("DESCRIBE FUNCTION log"), Row("Class: org.apache.spark.sql.catalyst.expressions.Logarithm") :: Row("Function: log") :: - Row("Usage: log(b, x) - Returns the logarithm of x with base b.") :: Nil + Row("Usage: log(base, expr) - Returns the logarithm of `expr` with `base`.") :: Nil ) // predicate operator checkAnswer( sql("DESCRIBE FUNCTION or"), Row("Class: org.apache.spark.sql.catalyst.expressions.Or") :: Row("Function: or") :: - Row("Usage: a or b - Logical OR.") :: Nil + Row("Usage: expr1 or expr2 - Logical OR.") :: Nil ) checkAnswer( sql("DESCRIBE FUNCTION !"), Row("Class: org.apache.spark.sql.catalyst.expressions.Not") :: Row("Function: !") :: - Row("Usage: ! a - Logical not") :: Nil + Row("Usage: ! expr - Logical not.") :: Nil ) // arithmetic operators checkAnswer( sql("DESCRIBE FUNCTION +"), Row("Class: org.apache.spark.sql.catalyst.expressions.Add") :: Row("Function: +") :: - Row("Usage: a + b - Returns a+b.") :: Nil + Row("Usage: expr1 + expr2 - Returns `expr1`+`expr2`.") :: Nil ) // comparison operators checkAnswer( sql("DESCRIBE FUNCTION <"), Row("Class: org.apache.spark.sql.catalyst.expressions.LessThan") :: Row("Function: <") :: - Row("Usage: a < b - Returns TRUE if a is less than b.") :: Nil + Row("Usage: expr1 < expr2 - Returns true if `expr1` is less than `expr2`.") :: Nil ) // STRING checkAnswer( @@ -1480,15 +1480,21 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { Row("Class: org.apache.spark.sql.catalyst.expressions.Concat") :: Row("Function: concat") :: Row("Usage: concat(str1, str2, ..., strN) " + - "- Returns the concatenation of str1, str2, ..., strN") :: Nil + "- Returns the concatenation of `str1`, `str2`, ..., `strN`.") :: Nil ) // extended mode checkAnswer( sql("DESCRIBE FUNCTION EXTENDED ^"), Row("Class: org.apache.spark.sql.catalyst.expressions.BitwiseXor") :: - Row("Extended Usage:\n> SELECT 3 ^ 5; 2") :: + Row( + """Extended Usage: + | Examples: + | > SELECT 3 ^ 5; + | 2 + | """.stripMargin) :: Row("Function: ^") :: - Row("Usage: a ^ b - Bitwise exclusive OR.") :: Nil + Row("Usage: expr1 ^ expr2 - Returns the result of " + + "bitwise exclusive OR of `expr1` and `expr2`.") :: Nil ) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 3a597d6afb15328e73b9d767af484051ddc5642f..ad70835d06d9267be7636208f0aa1d4fbcee7e0a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -271,15 +271,16 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { checkKeywordsExist(sql("describe function extended upper"), "Function: upper", "Class: org.apache.spark.sql.catalyst.expressions.Upper", - "Usage: upper(str) - Returns str with all characters changed to uppercase", + "Usage: upper(str) - Returns `str` with all characters changed to uppercase", "Extended Usage:", - "> SELECT upper('SparkSql')", - "'SPARKSQL'") + "Examples:", + "> SELECT upper('SparkSql');", + "SPARKSQL") checkKeywordsExist(sql("describe functioN Upper"), "Function: upper", "Class: org.apache.spark.sql.catalyst.expressions.Upper", - "Usage: upper(str) - Returns str with all characters changed to uppercase") + "Usage: upper(str) - Returns `str` with all characters changed to uppercase") checkKeywordsNotExist(sql("describe functioN Upper"), "Extended Usage") @@ -290,25 +291,28 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { checkKeywordsExist(sql("describe functioN `~`"), "Function: ~", "Class: org.apache.spark.sql.catalyst.expressions.BitwiseNot", - "Usage: ~ b - Bitwise NOT.") + "Usage: ~ expr - Returns the result of bitwise NOT of `expr`.") // Hard coded describe functions checkKeywordsExist(sql("describe function `<>`"), "Function: <>", - "Usage: a <> b - Returns TRUE if a is not equal to b") + "Usage: expr1 <> expr2 - Returns true if `expr1` is not equal to `expr2`") checkKeywordsExist(sql("describe function `!=`"), "Function: !=", - "Usage: a != b - Returns TRUE if a is not equal to b") + "Usage: expr1 != expr2 - Returns true if `expr1` is not equal to `expr2`") checkKeywordsExist(sql("describe function `between`"), "Function: between", - "Usage: a [NOT] BETWEEN b AND c - evaluate if a is [not] in between b and c") + "Usage: expr1 [NOT] BETWEEN expr2 AND expr3 - " + + "evaluate if `expr1` is [not] in between `expr2` and `expr3`") checkKeywordsExist(sql("describe function `case`"), "Function: case", - "Usage: CASE a WHEN b THEN c [WHEN d THEN e]* [ELSE f] END - " + - "When a = b, returns c; when a = d, return e; else return f") + "Usage: CASE expr1 WHEN expr2 THEN expr3 " + + "[WHEN expr4 THEN expr5]* [ELSE expr6] END - " + + "When `expr1` = `expr2`, returns `expr3`; " + + "when `expr1` = `expr4`, return `expr5`; else return `expr6`") } test("describe functions - user defined functions") {