Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -39,5 +39,5 @@
@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";
}
Original file line number Diff line number Diff line change
Expand Up @@ -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 {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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})"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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 {

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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) {

Expand All @@ -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
Expand All @@ -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
Expand All @@ -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
Expand All @@ -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"
Expand All @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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 {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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),
Expand All @@ -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),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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.")
Expand Down
Loading