Skip to content
Closed
Show file tree
Hide file tree
Changes from 3 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
34 changes: 34 additions & 0 deletions python/pyspark/sql/functions.py
Original file line number Diff line number Diff line change
Expand Up @@ -394,6 +394,23 @@ def cot(col):
return _invoke_function_over_column("cot", col)


def csc(col):
"""
.. versionadded:: 3.3.0
Copy link
Member

@HyukjinKwon HyukjinKwon Sep 17, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Shall we add a short description?

Copy link
Contributor Author

@yutoacts yutoacts Sep 17, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you, just added a description to all trig functions that missed it.


Parameters
----------
col : :class:`~pyspark.sql.Column` or str
Angle in radians

Returns
-------
:class:`~pyspark.sql.Column`
Cosecant of the angle.
"""
return _invoke_function_over_column("csc", col)


@since(1.4)
def exp(col):
"""
Expand Down Expand Up @@ -451,6 +468,23 @@ def rint(col):
return _invoke_function_over_column("rint", col)


def sec(col):
"""
.. versionadded:: 3.3.0

Parameters
----------
col : :class:`~pyspark.sql.Column` or str
Angle in radians

Returns
-------
:class:`~pyspark.sql.Column`
Secant of the angle.
"""
return _invoke_function_over_column("sec", col)


@since(1.4)
def signum(col):
"""
Expand Down
2 changes: 2 additions & 0 deletions python/pyspark/sql/functions.pyi
Original file line number Diff line number Diff line change
Expand Up @@ -299,6 +299,7 @@ def column(col: str) -> Column: ...
def cos(col: ColumnOrName) -> Column: ...
def cosh(col: ColumnOrName) -> Column: ...
def cot(col: ColumnOrName) -> Column: ...
def csc(col: ColumnOrName) -> Column: ...
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's better to let csc sit after count.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you for the review! fixed them.

def count(col: ColumnOrName) -> Column: ...
def cume_dist() -> Column: ...
def degrees(col: ColumnOrName) -> Column: ...
Expand Down Expand Up @@ -337,6 +338,7 @@ def rank() -> Column: ...
def rint(col: ColumnOrName) -> Column: ...
def row_number() -> Column: ...
def rtrim(col: ColumnOrName) -> Column: ...
def sec(col: ColumnOrName) -> Column: ...
def signum(col: ColumnOrName) -> Column: ...
def sin(col: ColumnOrName) -> Column: ...
def sinh(col: ColumnOrName) -> Column: ...
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -361,6 +361,7 @@ object FunctionRegistry {
expression[Ceil]("ceil"),
expression[Ceil]("ceiling", true),
expression[Cos]("cos"),
expression[Sec]("sec"),
expression[Cosh]("cosh"),
expression[Conv]("conv"),
expression[ToDegrees]("degrees"),
Expand Down Expand Up @@ -392,6 +393,7 @@ object FunctionRegistry {
expression[Signum]("sign", true),
expression[Signum]("signum"),
expression[Sin]("sin"),
expression[Csc]("csc"),
expression[Sinh]("sinh"),
expression[StringToMap]("str_to_map"),
expression[Sqrt]("sqrt"),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -299,6 +299,29 @@ case class Cos(child: Expression) extends UnaryMathExpression(math.cos, "COS") {
override protected def withNewChildInternal(newChild: Expression): Cos = copy(child = newChild)
}

@ExpressionDescription(
usage = """
_FUNC_(expr) - Returns the secant of `expr`, as if computed by `1/java.lang.Math.cos`.
""",
arguments = """
Arguments:
* expr - angle in radians
""",
examples = """
Examples:
> SELECT _FUNC_(0);
1.0
""",
since = "3.3.0",
group = "math_funcs")
case class Sec(child: Expression)
extends UnaryMathExpression((x: Double) => 1 / math.cos(x), "SEC") {
override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
defineCodeGen(ctx, ev, c => s"${ev.value} = 1 / java.lang.Math.cos($c);")
}
override protected def withNewChildInternal(newChild: Expression): Sec = copy(child = newChild)
}

@ExpressionDescription(
usage = """
_FUNC_(expr) - Returns the hyperbolic cosine of `expr`, as if computed by
Expand Down Expand Up @@ -655,6 +678,29 @@ case class Sin(child: Expression) extends UnaryMathExpression(math.sin, "SIN") {
override protected def withNewChildInternal(newChild: Expression): Sin = copy(child = newChild)
}

@ExpressionDescription(
usage = """
_FUNC_(expr) - Returns the cosecant of `expr`, as if computed by `1/java.lang.Math.sin`.
""",
arguments = """
Arguments:
* expr - angle in radians
""",
examples = """
Examples:
> SELECT _FUNC_(1);
1.1883951057781212
""",
since = "3.3.0",
group = "math_funcs")
case class Csc(child: Expression)
extends UnaryMathExpression((x: Double) => 1 / math.sin(x), "CSC") {
override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
defineCodeGen(ctx, ev, c => s"${ev.value} = 1 / java.lang.Math.sin($c);")
}
override protected def withNewChildInternal(newChild: Expression): Csc = copy(child = newChild)
}

@ExpressionDescription(
usage = """
_FUNC_(expr) - Returns hyperbolic sine of `expr`, as if computed by `java.lang.Math._FUNC_`.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -187,6 +187,20 @@ class MathExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
checkConsistencyBetweenInterpretedAndCodegen(Sin, DoubleType)
}

test("csc") {
def f: (Double) => Double = (x: Double) => 1 / math.sin(x)
testUnary(Csc, f)
checkConsistencyBetweenInterpretedAndCodegen(Csc, DoubleType)
val nullLit = Literal.create(null, NullType)
val intNullLit = Literal.create(null, IntegerType)
val intLit = Literal.create(1, IntegerType)
checkEvaluation(checkDataTypeAndCast(Csc(nullLit)), null, EmptyRow)
checkEvaluation(checkDataTypeAndCast(Csc(intNullLit)), null, EmptyRow)
checkEvaluation(checkDataTypeAndCast(Csc(intLit)), 1 / math.sin(1), EmptyRow)
checkEvaluation(checkDataTypeAndCast(Csc(-intLit)), 1 / math.sin(-1), EmptyRow)
checkEvaluation(checkDataTypeAndCast(Csc(0)), 1 / math.sin(0), EmptyRow)
}

test("asin") {
testUnary(Asin, math.asin, (-10 to 10).map(_ * 0.1))
testUnary(Asin, math.asin, (11 to 20).map(_ * 0.1), expectNaN = true)
Expand Down Expand Up @@ -215,6 +229,20 @@ class MathExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
checkConsistencyBetweenInterpretedAndCodegen(Cos, DoubleType)
}

test("sec") {
def f: (Double) => Double = (x: Double) => 1 / math.cos(x)
testUnary(Sec, f)
checkConsistencyBetweenInterpretedAndCodegen(Sec, DoubleType)
val nullLit = Literal.create(null, NullType)
val intNullLit = Literal.create(null, IntegerType)
val intLit = Literal.create(1, IntegerType)
checkEvaluation(checkDataTypeAndCast(Sec(nullLit)), null, EmptyRow)
checkEvaluation(checkDataTypeAndCast(Sec(intNullLit)), null, EmptyRow)
checkEvaluation(checkDataTypeAndCast(Sec(intLit)), 1 / math.cos(1), EmptyRow)
checkEvaluation(checkDataTypeAndCast(Sec(-intLit)), 1 / math.cos(-1), EmptyRow)
checkEvaluation(checkDataTypeAndCast(Sec(0)), 1 / math.cos(0), EmptyRow)
}

test("acos") {
testUnary(Acos, math.acos, (-10 to 10).map(_ * 0.1))
testUnary(Acos, math.acos, (11 to 20).map(_ * 0.1), expectNaN = true)
Expand Down
18 changes: 18 additions & 0 deletions sql/core/src/main/scala/org/apache/spark/sql/functions.scala
Original file line number Diff line number Diff line change
Expand Up @@ -1809,6 +1809,15 @@ object functions {
*/
def cot(e: Column): Column = withExpr { Cot(e.expr) }

/**
* @param e angle in radians
* @return cosecant of the angle
*
* @group math_funcs
* @since 3.3.0
*/
def csc(e: Column): Column = withExpr { Csc(e.expr) }

/**
* Computes the exponential of the given value.
*
Expand Down Expand Up @@ -2197,6 +2206,15 @@ object functions {
*/
def bround(e: Column, scale: Int): Column = withExpr { BRound(e.expr, Literal(scale)) }

/**
* @param e angle in radians
* @return secant of the angle
*
* @group math_funcs
* @since 3.3.0
*/
def sec(e: Column): Column = withExpr { Sec(e.expr) }

/**
* Shift the given value numBits left. If the given value is a long value, this function
* will return a long value else it will return an integer value.
Expand Down
8 changes: 8 additions & 0 deletions sql/core/src/test/resources/sql-tests/inputs/operators.sql
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,14 @@ select 5 % 3;
select pmod(-7, 3);

-- math functions
select sec(1);
select sec(null);
select sec(0);
select sec(-1);
select csc(1);
select csc(null);
select csc(0);
select csc(-1);:
select cot(1);
select cot(null);
select cot(0);
Expand Down
78 changes: 75 additions & 3 deletions sql/core/src/test/resources/sql-tests/results/operators.sql.out
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
-- Automatically generated by SQLQueryTestSuite
-- Number of queries: 69
-- Number of queries: 77


-- !query
Expand Down Expand Up @@ -259,11 +259,83 @@ struct<pmod(-7, 3):int>


-- !query
select sec(1)
-- !query schema
struct<SEC(1):double>
-- !query output
1.8508157176809255


-- !query
select sec(null)
-- !query schema
struct<SEC(NULL):double>
-- !query output
NULL


-- !query
select sec(0)
-- !query schema
struct<SEC(0):double>
-- !query output
1.0


-- !query
select sec(-1)
-- !query schema
struct<SEC(-1):double>
-- !query output
1.8508157176809255


-- !query
select csc(1)
-- !query schema
struct<CSC(1):double>
-- !query output
1.1883951057781212


-- !query
select csc(null)
-- !query schema
struct<CSC(NULL):double>
-- !query output
NULL


-- !query
select csc(0)
-- !query schema
struct<CSC(0):double>
-- !query output
Infinity


-- !query
select csc(-1)
-- !query schema
struct<CSC(-1):double>
-- !query output
-1.1883951057781212


-- !query
:
select cot(1)
-- !query schema
struct<COT(1):double>
struct<>
-- !query output
0.6420926159343306
org.apache.spark.sql.catalyst.parser.ParseException

extraneous input ':' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMENT', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MERGE', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0)

== SQL ==
:
^^^
select cot(1)


-- !query
Expand Down