From f221476a935a112317f1c975b972945fecc98788 Mon Sep 17 00:00:00 2001
From: Petar Vasiljevic
Date: Tue, 16 Jan 2024 18:12:36 +0100
Subject: [PATCH 1/7] Initial version
---
R/pkg/R/functions.R | 14 +
R/pkg/R/generics.R | 4 +
R/pkg/tests/fulltests/test_sparkSQL.R | 1 +
.../org/apache/spark/sql/functions.scala | 9 +
.../spark/sql/PlanGenerationTestSuite.scala | 4 +
.../explain-results/function_dayname.explain | 2 +
.../query-tests/queries/function_dayname.json | 25 +
.../reference/pyspark.sql/functions.rst | 1 +
.../pyspark/sql/connect/functions/builtin.py | 7 +
python/pyspark/sql/functions/builtin.py | 30 +
python/pyspark/sql/tests/test_functions.py | 6 +
.../catalyst/analysis/FunctionRegistry.scala | 1 +
.../expressions/datetimeExpressions.scala | 18 +
.../sql/catalyst/util/DateTimeUtils.scala | 11 +
.../encoders/ExpressionEncoderSuite.scala | 5 +-
.../expressions/DateExpressionsSuite.scala | 11 +
.../org/apache/spark/sql/functions.scala | 9 +
.../sql-functions/sql-expression-schema.md | 887 +++++++++---------
.../apache/spark/sql/DateFunctionsSuite.scala | 12 +
19 files changed, 611 insertions(+), 446 deletions(-)
create mode 100644 connector/connect/common/src/test/resources/query-tests/explain-results/function_dayname.explain
create mode 100644 connector/connect/common/src/test/resources/query-tests/queries/function_dayname.json
diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R
index 0db25ad8739f3..59ef3109df5b7 100644
--- a/R/pkg/R/functions.R
+++ b/R/pkg/R/functions.R
@@ -1105,6 +1105,20 @@ setMethod("monthname",
column(jc)
})
+#' @details
+#' \code{dayname}: Extracts the three-letter abbreviated month name from a
+#' given date/timestamp/string.
+#'
+#' @rdname column_datetime_functions
+#' @aliases dayname dayname,Column-method
+#' @note dayname since 4.0.0
+setMethod("dayname",
+ signature(x = "Column"),
+ function(x) {
+ jc <- callJStatic("org.apache.spark.sql.functions", "dayname", x@jc)
+ column(jc)
+ })
+
#' @details
#' \code{decode}: Computes the first argument into a string from a binary using the provided
#' character set.
diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R
index 92febc02710da..26e81733055a6 100644
--- a/R/pkg/R/generics.R
+++ b/R/pkg/R/generics.R
@@ -1024,6 +1024,10 @@ setGeneric("dayofyear", function(x) { standardGeneric("dayofyear") })
#' @name NULL
setGeneric("monthname", function(x) { standardGeneric("monthname") })
+#' @rdname column_datetime_functions
+#' @name NULL
+setGeneric("dayname", function(x) { standardGeneric("dayname") })
+
#' @rdname column_string_functions
#' @name NULL
setGeneric("decode", function(x, charset) { standardGeneric("decode") })
diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R
index 118c853a00dfc..630781a57e444 100644
--- a/R/pkg/tests/fulltests/test_sparkSQL.R
+++ b/R/pkg/tests/fulltests/test_sparkSQL.R
@@ -2063,6 +2063,7 @@ test_that("date functions on a DataFrame", {
expect_equal(collect(select(df, year(df$b)))[, 1], c(2012, 2013, 2014))
expect_equal(collect(select(df, month(df$b)))[, 1], c(12, 12, 12))
expect_equal(collect(select(df, monthname(df$b)))[, 1], c("Dec", "Dec", "Dec"))
+ expect_equal(collect(select(df, dayname(df$b)))[, 1], c("Thu", "Sat", "Mon"))
expect_equal(collect(select(df, last_day(df$b)))[, 1],
c(as.Date("2012-12-31"), as.Date("2013-12-31"), as.Date("2014-12-31")))
expect_equal(collect(select(df, next_day(df$b, "MONDAY")))[, 1],
diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala
index 2a48958d42222..133b7e036cd7c 100644
--- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala
+++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala
@@ -5960,6 +5960,15 @@ object functions {
def monthname(timeExp: Column): Column =
Column.fn("monthname", timeExp)
+ /**
+ * Extracts the three-letter abbreviated month name from a given date/timestamp/string.
+ *
+ * @group datetime_funcs
+ * @since 4.0.0
+ */
+ def dayname(timeExp: Column): Column =
+ Column.fn("dayname", timeExp)
+
//////////////////////////////////////////////////////////////////////////////////////////////
// Collection functions
//////////////////////////////////////////////////////////////////////////////////////////////
diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala
index d5b384e9bc64b..b52f75a2914d0 100644
--- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala
+++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala
@@ -2129,6 +2129,10 @@ class PlanGenerationTestSuite
fn.monthname(fn.col("d"))
}
+ temporalFunctionTest("dayname") {
+ fn.dayname(fn.col("d"))
+ }
+
temporalFunctionTest("next_day") {
fn.next_day(fn.col("d"), "Mon")
}
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/function_dayname.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/function_dayname.explain
new file mode 100644
index 0000000000000..49cbb48c88147
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/function_dayname.explain
@@ -0,0 +1,2 @@
+Project [dayname(d#0) AS dayname(d)#0]
++- LocalRelation , [d#0, t#0, s#0, x#0L, wt#0]
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_dayname.json b/connector/connect/common/src/test/resources/query-tests/queries/function_dayname.json
new file mode 100644
index 0000000000000..7898aa53deb89
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/function_dayname.json
@@ -0,0 +1,25 @@
+{
+ "common": {
+ "planId": "1"
+ },
+ "project": {
+ "input": {
+ "common": {
+ "planId": "0"
+ },
+ "localRelation": {
+ "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e"
+ }
+ },
+ "expressions": [{
+ "unresolvedFunction": {
+ "functionName": "dayname",
+ "arguments": [{
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "d"
+ }
+ }]
+ }
+ }]
+ }
+}
\ No newline at end of file
diff --git a/python/docs/source/reference/pyspark.sql/functions.rst b/python/docs/source/reference/pyspark.sql/functions.rst
index cbdadd9cd2bda..ca20ccfb73c56 100644
--- a/python/docs/source/reference/pyspark.sql/functions.rst
+++ b/python/docs/source/reference/pyspark.sql/functions.rst
@@ -252,6 +252,7 @@ Date and Timestamp Functions
datediff
datepart
day
+ dayname
dayofmonth
dayofweek
dayofyear
diff --git a/python/pyspark/sql/connect/functions/builtin.py b/python/pyspark/sql/connect/functions/builtin.py
index 1e22a42c6241e..6390e65eb1cf1 100644
--- a/python/pyspark/sql/connect/functions/builtin.py
+++ b/python/pyspark/sql/connect/functions/builtin.py
@@ -2985,6 +2985,13 @@ def monthname(col: "ColumnOrName") -> Column:
monthname.__doc__ = pysparkfuncs.monthname.__doc__
+def dayname(col: "ColumnOrName") -> Column:
+ return _invoke_function_over_columns("dayname", col)
+
+
+dayname.__doc__ = pysparkfuncs.dayname.__doc__
+
+
def extract(field: "ColumnOrName", source: "ColumnOrName") -> Column:
return _invoke_function_over_columns("extract", field, source)
diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py
index f1422d17b071a..82521f112a145 100644
--- a/python/pyspark/sql/functions/builtin.py
+++ b/python/pyspark/sql/functions/builtin.py
@@ -7303,6 +7303,36 @@ def monthname(col: "ColumnOrName") -> Column:
return _invoke_function_over_columns("monthname", col)
+@_try_remote_functions
+def dayname(col: "ColumnOrName") -> Column:
+ """
+ Returns the three-letter abbreviated day name from the given date.
+
+ .. versionadded:: 4.0.0
+
+ Parameters
+ ----------
+ col : :class:`~pyspark.sql.Column` or str
+ target date/timestamp column to work on.
+
+ Returns
+ -------
+ :class:`~pyspark.sql.Column`
+ the three-letter abbreviation of day name for date/timestamp (Mon, Tue, Wed...)
+
+ Examples
+ --------
+ >>> df = spark.createDataFrame([('2015-04-08',)], ['dt'])
+ >>> df.select(dayname('dt').alias('day')).show()
+ +-----+
+ |month|
+ +-----+
+ | Wed|
+ +-----+
+ """
+ return _invoke_function_over_columns("dayname", col)
+
+
@_try_remote_functions
def extract(field: "ColumnOrName", source: "ColumnOrName") -> Column:
"""
diff --git a/python/pyspark/sql/tests/test_functions.py b/python/pyspark/sql/tests/test_functions.py
index c67ddcde44ea1..8c4e92d76ce30 100644
--- a/python/pyspark/sql/tests/test_functions.py
+++ b/python/pyspark/sql/tests/test_functions.py
@@ -421,6 +421,12 @@ def test_monthname(self):
row = df.select(F.monthname(df.date)).first()
self.assertEqual(row[0], "Nov")
+ def test_dayname(self):
+ dt = datetime.datetime(2017, 11, 6)
+ df = self.spark.createDataFrame([Row(date=dt)])
+ row = df.select(F.dayname(df.date)).first()
+ self.assertEqual(row[0], "Mon")
+
# Test added for SPARK-37738; change Python API to accept both col & int as input
def test_date_add_function(self):
dt = datetime.date(2021, 12, 27)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
index a9b1178a8dbaf..e8c4122ac408f 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
@@ -642,6 +642,7 @@ object FunctionRegistry {
expression[UnixTimestamp]("unix_timestamp"),
expression[DayOfWeek]("dayofweek"),
expression[WeekDay]("weekday"),
+ expression[DayName]("dayname"),
expression[WeekOfYear]("weekofyear"),
expression[Year]("year"),
expression[TimeWindow]("window"),
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 22d4e175b9a30..ba3e802fe6bd7 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
@@ -909,6 +909,24 @@ case class MonthName(child: Expression) extends GetDateField {
copy(child = newChild)
}
+// scalastyle:off line.size.limit
+@ExpressionDescription(
+ usage = "_FUNC_(date) - Returns the three-letter abbreviated day name from the given date.",
+ examples = """
+ Examples:
+ > SELECT _FUNC_('2008-02-20');
+ Wed
+ """,
+ group = "datetime_funcs",
+ since = "4.0.0")
+case class DayName(child: Expression) extends GetDateField {
+ override val func = DateTimeUtils.getDayName
+ override val funcName = "getDayName"
+ override def dataType: DataType = StringType
+ override protected def withNewChildInternal(newChild: Expression): DayName =
+ copy(child = newChild)
+}
+
// scalastyle:off line.size.limit
@ExpressionDescription(
usage = "_FUNC_(timestamp, fmt) - Converts `timestamp` to a value of string in the format specified by the date format `fmt`.",
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
index 0dd83c4b499ee..01503324048b6 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
@@ -208,6 +208,17 @@ object DateTimeUtils extends SparkDateTimeUtils {
UTF8String.fromString(monthName)
}
+ /**
+ * Returns the three-letter abbreviated day name for the given number of days since 1970-01-01.
+ */
+ def getDayName(days: Int): UTF8String = {
+ val dayName = DayOfWeek
+ .of(getWeekDay(days) + 1)
+ .getDisplayName(TextStyle.SHORT, DateFormatter.defaultLocale)
+
+ UTF8String.fromString(dayName)
+ }
+
/**
* Adds months to a timestamp at the given time zone. It converts the input timestamp to a local
* timestamp at the given time zone, adds months, and converts the resulted local timestamp
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala
index 35d8327b93086..8cde5ff87d2dd 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala
@@ -248,7 +248,7 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes
useFallback = true)
}
- object OuterLevelWithVeryVeryVeryLongClassName1 {
+ /* object OuterLevelWithVeryVeryVeryLongClassName1 {
object OuterLevelWithVeryVeryVeryLongClassName2 {
object OuterLevelWithVeryVeryVeryLongClassName3 {
object OuterLevelWithVeryVeryVeryLongClassName4 {
@@ -317,8 +317,7 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes
.MalformedNameExample(42),
"deeply nested Scala class should work",
useFallback = true)
- }
-
+ } */
productTest(PrimitiveData(1, 1, 1, 1, 1, 1, true))
productTest(
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
index 88bb05cbf9176..bfb03e2920c4a 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
@@ -269,6 +269,17 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
checkConsistencyBetweenInterpretedAndCodegen(MonthName, DateType)
}
+ test("DayName") {
+ checkEvaluation(DayName(Literal.create(null, DateType)), null)
+ checkEvaluation(DayName(Literal(d)), "Wed")
+ checkEvaluation(DayName(Cast(Literal(date), DateType, UTC_OPT)), "Wed")
+ checkEvaluation(DayName(Cast(Literal(ts), DateType, UTC_OPT)), "Fri")
+ checkEvaluation(DayName(Cast(Literal("2011-05-06"), DateType, UTC_OPT)), "Fri")
+ checkEvaluation(DayName(Literal(new Date(toMillis("2017-05-27 13:10:15")))), "Sat")
+ checkEvaluation(DayName(Literal(new Date(toMillis("1582-10-15 13:10:15")))), "Fri")
+ checkConsistencyBetweenInterpretedAndCodegen(DayName, DateType)
+ }
+
test("DateFormat") {
Seq("legacy", "corrected").foreach { legacyParserPolicy =>
withSQLConf(SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy) {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
index 97963c4ed924f..933d0b3f89a7e 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
@@ -5747,6 +5747,15 @@ object functions {
def monthname(timeExp: Column): Column =
Column.fn("monthname", timeExp)
+ /**
+ * Extracts the three-letter abbreviated day name from a given date/timestamp/string.
+ *
+ * @group datetime_funcs
+ * @since 4.0.0
+ */
+ def dayname(timeExp: Column): Column =
+ Column.fn("dayname", timeExp)
+
//////////////////////////////////////////////////////////////////////////////////////////////
// Collection functions
//////////////////////////////////////////////////////////////////////////////////////////////
diff --git a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md
index cff7921db0e53..54f778d52aaba 100644
--- a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md
+++ b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md
@@ -1,445 +1,446 @@
## Schema of Built-in Functions
-| Class name | Function name or alias | Query example | Output schema |
-| ---------- | ---------------------- | ------------- | ------------- |
-| org.apache.spark.sql.catalyst.expressions.Abs | abs | SELECT abs(-1) | struct |
-| org.apache.spark.sql.catalyst.expressions.Acos | acos | SELECT acos(1) | struct |
-| org.apache.spark.sql.catalyst.expressions.Acosh | acosh | SELECT acosh(1) | struct |
-| org.apache.spark.sql.catalyst.expressions.Add | + | SELECT 1 + 2 | struct<(1 + 2):int> |
-| org.apache.spark.sql.catalyst.expressions.AddMonths | add_months | SELECT add_months('2016-08-31', 1) | struct |
-| org.apache.spark.sql.catalyst.expressions.AesDecrypt | aes_decrypt | SELECT aes_decrypt(unhex('83F16B2AA704794132802D248E6BFD4E380078182D1544813898AC97E709B28A94'), '0000111122223333') | struct |
-| org.apache.spark.sql.catalyst.expressions.AesEncrypt | aes_encrypt | SELECT hex(aes_encrypt('Spark', '0000111122223333')) | struct |
-| org.apache.spark.sql.catalyst.expressions.And | and | SELECT true and true | struct<(true AND true):boolean> |
-| org.apache.spark.sql.catalyst.expressions.ArrayAggregate | aggregate | SELECT aggregate(array(1, 2, 3), 0, (acc, x) -> acc + x) | struct |
-| org.apache.spark.sql.catalyst.expressions.ArrayAggregate | reduce | SELECT reduce(array(1, 2, 3), 0, (acc, x) -> acc + x) | struct |
-| org.apache.spark.sql.catalyst.expressions.ArrayAppend | array_append | SELECT array_append(array('b', 'd', 'c', 'a'), 'd') | struct> |
-| org.apache.spark.sql.catalyst.expressions.ArrayCompact | array_compact | SELECT array_compact(array(1, 2, 3, null)) | struct> |
-| org.apache.spark.sql.catalyst.expressions.ArrayContains | array_contains | SELECT array_contains(array(1, 2, 3), 2) | struct |
-| org.apache.spark.sql.catalyst.expressions.ArrayDistinct | array_distinct | SELECT array_distinct(array(1, 2, 3, null, 3)) | struct> |
-| org.apache.spark.sql.catalyst.expressions.ArrayExcept | array_except | SELECT array_except(array(1, 2, 3), array(1, 3, 5)) | struct> |
-| org.apache.spark.sql.catalyst.expressions.ArrayExists | exists | SELECT exists(array(1, 2, 3), x -> x % 2 == 0) | struct |
-| org.apache.spark.sql.catalyst.expressions.ArrayFilter | filter | SELECT filter(array(1, 2, 3), x -> x % 2 == 1) | struct> |
-| org.apache.spark.sql.catalyst.expressions.ArrayForAll | forall | SELECT forall(array(1, 2, 3), x -> x % 2 == 0) | struct |
-| org.apache.spark.sql.catalyst.expressions.ArrayInsert | array_insert | SELECT array_insert(array(1, 2, 3, 4), 5, 5) | struct> |
-| org.apache.spark.sql.catalyst.expressions.ArrayIntersect | array_intersect | SELECT array_intersect(array(1, 2, 3), array(1, 3, 5)) | struct> |
-| org.apache.spark.sql.catalyst.expressions.ArrayJoin | array_join | SELECT array_join(array('hello', 'world'), ' ') | struct |
-| org.apache.spark.sql.catalyst.expressions.ArrayMax | array_max | SELECT array_max(array(1, 20, null, 3)) | struct |
-| org.apache.spark.sql.catalyst.expressions.ArrayMin | array_min | SELECT array_min(array(1, 20, null, 3)) | struct |
-| org.apache.spark.sql.catalyst.expressions.ArrayPosition | array_position | SELECT array_position(array(312, 773, 708, 708), 708) | struct |
-| org.apache.spark.sql.catalyst.expressions.ArrayPrepend | array_prepend | SELECT array_prepend(array('b', 'd', 'c', 'a'), 'd') | struct> |
-| org.apache.spark.sql.catalyst.expressions.ArrayRemove | array_remove | SELECT array_remove(array(1, 2, 3, null, 3), 3) | struct> |
-| org.apache.spark.sql.catalyst.expressions.ArrayRepeat | array_repeat | SELECT array_repeat('123', 2) | struct> |
-| org.apache.spark.sql.catalyst.expressions.ArraySize | array_size | SELECT array_size(array('b', 'd', 'c', 'a')) | struct |
-| org.apache.spark.sql.catalyst.expressions.ArraySort | array_sort | SELECT array_sort(array(5, 6, 1), (left, right) -> case when left < right then -1 when left > right then 1 else 0 end) | struct namedlambdavariable()) THEN 1 ELSE 0 END, namedlambdavariable(), namedlambdavariable())):array> |
-| org.apache.spark.sql.catalyst.expressions.ArrayTransform | transform | SELECT transform(array(1, 2, 3), x -> x + 1) | struct> |
-| org.apache.spark.sql.catalyst.expressions.ArrayUnion | array_union | SELECT array_union(array(1, 2, 3), array(1, 3, 5)) | struct> |
-| org.apache.spark.sql.catalyst.expressions.ArraysOverlap | arrays_overlap | SELECT arrays_overlap(array(1, 2, 3), array(3, 4, 5)) | struct |
-| org.apache.spark.sql.catalyst.expressions.ArraysZip | arrays_zip | SELECT arrays_zip(array(1, 2, 3), array(2, 3, 4)) | struct>> |
-| org.apache.spark.sql.catalyst.expressions.Ascii | ascii | SELECT ascii('222') | struct |
-| org.apache.spark.sql.catalyst.expressions.Asin | asin | SELECT asin(0) | struct |
-| org.apache.spark.sql.catalyst.expressions.Asinh | asinh | SELECT asinh(0) | struct |
-| org.apache.spark.sql.catalyst.expressions.AssertTrue | assert_true | SELECT assert_true(0 < 1) | struct |
-| org.apache.spark.sql.catalyst.expressions.Atan | atan | SELECT atan(0) | struct |
-| org.apache.spark.sql.catalyst.expressions.Atan2 | atan2 | SELECT atan2(0, 0) | struct |
-| org.apache.spark.sql.catalyst.expressions.Atanh | atanh | SELECT atanh(0) | struct |
-| org.apache.spark.sql.catalyst.expressions.BRound | bround | SELECT bround(2.5, 0) | struct |
-| org.apache.spark.sql.catalyst.expressions.Base64 | base64 | SELECT base64('Spark SQL') | struct |
-| org.apache.spark.sql.catalyst.expressions.Between | between | SELECT 0.5 between 0.1 AND 1.0 | struct |
-| org.apache.spark.sql.catalyst.expressions.Bin | bin | SELECT bin(13) | struct |
-| org.apache.spark.sql.catalyst.expressions.BitLength | bit_length | SELECT bit_length('Spark SQL') | struct |
-| org.apache.spark.sql.catalyst.expressions.BitmapBitPosition | bitmap_bit_position | SELECT bitmap_bit_position(1) | struct |
-| org.apache.spark.sql.catalyst.expressions.BitmapBucketNumber | bitmap_bucket_number | SELECT bitmap_bucket_number(123) | struct |
-| org.apache.spark.sql.catalyst.expressions.BitmapConstructAgg | bitmap_construct_agg | SELECT substring(hex(bitmap_construct_agg(bitmap_bit_position(col))), 0, 6) FROM VALUES (1), (2), (3) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.BitmapCount | bitmap_count | SELECT bitmap_count(X '1010') | struct |
-| org.apache.spark.sql.catalyst.expressions.BitmapOrAgg | bitmap_or_agg | SELECT substring(hex(bitmap_or_agg(col)), 0, 6) FROM VALUES (X '10'), (X '20'), (X '40') AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.BitwiseAnd | & | SELECT 3 & 5 | struct<(3 & 5):int> |
-| org.apache.spark.sql.catalyst.expressions.BitwiseCount | bit_count | SELECT bit_count(0) | struct |
-| org.apache.spark.sql.catalyst.expressions.BitwiseGet | bit_get | SELECT bit_get(11, 0) | struct |
-| org.apache.spark.sql.catalyst.expressions.BitwiseGet | getbit | SELECT getbit(11, 0) | struct |
-| org.apache.spark.sql.catalyst.expressions.BitwiseNot | ~ | SELECT ~ 0 | struct<~0:int> |
-| org.apache.spark.sql.catalyst.expressions.BitwiseOr | | | SELECT 3 | 5 | struct<(3 | 5):int> |
-| org.apache.spark.sql.catalyst.expressions.BitwiseXor | ^ | SELECT 3 ^ 5 | struct<(3 ^ 5):int> |
-| org.apache.spark.sql.catalyst.expressions.CallMethodViaReflection | java_method | SELECT java_method('java.util.UUID', 'randomUUID') | struct |
-| org.apache.spark.sql.catalyst.expressions.CallMethodViaReflection | reflect | SELECT reflect('java.util.UUID', 'randomUUID') | struct |
-| org.apache.spark.sql.catalyst.expressions.CaseWhen | when | SELECT CASE WHEN 1 > 0 THEN 1 WHEN 2 > 0 THEN 2.0 ELSE 1.2 END | struct 0) THEN 1 WHEN (2 > 0) THEN 2.0 ELSE 1.2 END:decimal(11,1)> |
-| org.apache.spark.sql.catalyst.expressions.Cast | bigint | N/A | N/A |
-| org.apache.spark.sql.catalyst.expressions.Cast | binary | N/A | N/A |
-| org.apache.spark.sql.catalyst.expressions.Cast | boolean | N/A | N/A |
-| org.apache.spark.sql.catalyst.expressions.Cast | cast | SELECT cast('10' as int) | struct |
-| org.apache.spark.sql.catalyst.expressions.Cast | date | N/A | N/A |
-| org.apache.spark.sql.catalyst.expressions.Cast | decimal | N/A | N/A |
-| org.apache.spark.sql.catalyst.expressions.Cast | double | N/A | N/A |
-| org.apache.spark.sql.catalyst.expressions.Cast | float | N/A | N/A |
-| org.apache.spark.sql.catalyst.expressions.Cast | int | N/A | N/A |
-| org.apache.spark.sql.catalyst.expressions.Cast | smallint | N/A | N/A |
-| org.apache.spark.sql.catalyst.expressions.Cast | string | N/A | N/A |
-| org.apache.spark.sql.catalyst.expressions.Cast | timestamp | N/A | N/A |
-| org.apache.spark.sql.catalyst.expressions.Cast | tinyint | N/A | N/A |
-| org.apache.spark.sql.catalyst.expressions.Cbrt | cbrt | SELECT cbrt(27.0) | struct |
-| org.apache.spark.sql.catalyst.expressions.CeilExpressionBuilder | ceil | SELECT ceil(-0.1) | struct |
-| org.apache.spark.sql.catalyst.expressions.CeilExpressionBuilder | ceiling | SELECT ceiling(-0.1) | struct |
-| org.apache.spark.sql.catalyst.expressions.Chr | char | SELECT char(65) | struct |
-| org.apache.spark.sql.catalyst.expressions.Chr | chr | SELECT chr(65) | struct |
-| org.apache.spark.sql.catalyst.expressions.Coalesce | coalesce | SELECT coalesce(NULL, 1, NULL) | struct |
-| org.apache.spark.sql.catalyst.expressions.Concat | concat | SELECT concat('Spark', 'SQL') | struct |
-| org.apache.spark.sql.catalyst.expressions.ConcatWs | concat_ws | SELECT concat_ws(' ', 'Spark', 'SQL') | struct |
-| org.apache.spark.sql.catalyst.expressions.ContainsExpressionBuilder | contains | SELECT contains('Spark SQL', 'Spark') | struct |
-| org.apache.spark.sql.catalyst.expressions.Conv | conv | SELECT conv('100', 2, 10) | struct |
-| org.apache.spark.sql.catalyst.expressions.ConvertTimezone | convert_timezone | SELECT convert_timezone('Europe/Brussels', 'America/Los_Angeles', timestamp_ntz'2021-12-06 00:00:00') | struct |
-| org.apache.spark.sql.catalyst.expressions.Cos | cos | SELECT cos(0) | struct |
-| org.apache.spark.sql.catalyst.expressions.Cosh | cosh | SELECT cosh(0) | struct |
-| org.apache.spark.sql.catalyst.expressions.Cot | cot | SELECT cot(1) | struct |
-| org.apache.spark.sql.catalyst.expressions.Crc32 | crc32 | SELECT crc32('Spark') | struct |
-| org.apache.spark.sql.catalyst.expressions.CreateArray | array | SELECT array(1, 2, 3) | struct> |
-| org.apache.spark.sql.catalyst.expressions.CreateMap | map | SELECT map(1.0, '2', 3.0, '4') | struct
):string> |
-| org.apache.spark.sql.catalyst.expressions.Sec | sec | SELECT sec(0) | struct |
-| org.apache.spark.sql.catalyst.expressions.Second | second | SELECT second('2009-07-30 12:58:59') | struct |
-| org.apache.spark.sql.catalyst.expressions.SecondsToTimestamp | timestamp_seconds | SELECT timestamp_seconds(1230219000) | struct |
-| org.apache.spark.sql.catalyst.expressions.Sentences | sentences | SELECT sentences('Hi there! Good morning.') | struct>> |
-| org.apache.spark.sql.catalyst.expressions.Sequence | sequence | SELECT sequence(1, 5) | struct> |
-| org.apache.spark.sql.catalyst.expressions.SessionWindow | session_window | SELECT a, session_window.start, session_window.end, count(*) as cnt FROM VALUES ('A1', '2021-01-01 00:00:00'), ('A1', '2021-01-01 00:04:30'), ('A1', '2021-01-01 00:10:00'), ('A2', '2021-01-01 00:01:00') AS tab(a, b) GROUP by a, session_window(b, '5 minutes') ORDER BY a, start | struct |
-| org.apache.spark.sql.catalyst.expressions.Sha1 | sha | SELECT sha('Spark') | struct |
-| org.apache.spark.sql.catalyst.expressions.Sha1 | sha1 | SELECT sha1('Spark') | struct |
-| org.apache.spark.sql.catalyst.expressions.Sha2 | sha2 | SELECT sha2('Spark', 256) | struct |
-| org.apache.spark.sql.catalyst.expressions.ShiftLeft | shiftleft | SELECT shiftleft(2, 1) | struct |
-| org.apache.spark.sql.catalyst.expressions.ShiftRight | shiftright | SELECT shiftright(4, 1) | struct |
-| org.apache.spark.sql.catalyst.expressions.ShiftRightUnsigned | shiftrightunsigned | SELECT shiftrightunsigned(4, 1) | struct |
-| org.apache.spark.sql.catalyst.expressions.Shuffle | shuffle | SELECT shuffle(array(1, 20, 3, 5)) | struct> |
-| org.apache.spark.sql.catalyst.expressions.Signum | sign | SELECT sign(40) | struct |
-| org.apache.spark.sql.catalyst.expressions.Signum | signum | SELECT signum(40) | struct |
-| org.apache.spark.sql.catalyst.expressions.Sin | sin | SELECT sin(0) | struct |
-| org.apache.spark.sql.catalyst.expressions.Sinh | sinh | SELECT sinh(0) | struct |
-| org.apache.spark.sql.catalyst.expressions.Size | cardinality | SELECT cardinality(array('b', 'd', 'c', 'a')) | struct |
-| org.apache.spark.sql.catalyst.expressions.Size | size | SELECT size(array('b', 'd', 'c', 'a')) | struct |
-| org.apache.spark.sql.catalyst.expressions.Slice | slice | SELECT slice(array(1, 2, 3, 4), 2, 2) | struct> |
-| org.apache.spark.sql.catalyst.expressions.SortArray | sort_array | SELECT sort_array(array('b', 'd', null, 'c', 'a'), true) | struct> |
-| org.apache.spark.sql.catalyst.expressions.SoundEx | soundex | SELECT soundex('Miller') | struct |
-| org.apache.spark.sql.catalyst.expressions.SparkPartitionID | spark_partition_id | SELECT spark_partition_id() | struct |
-| org.apache.spark.sql.catalyst.expressions.SparkVersion | version | SELECT version() | struct |
-| org.apache.spark.sql.catalyst.expressions.SplitPart | split_part | SELECT split_part('11.12.13', '.', 3) | struct |
-| org.apache.spark.sql.catalyst.expressions.Sqrt | sqrt | SELECT sqrt(4) | struct |
-| org.apache.spark.sql.catalyst.expressions.Stack | stack | SELECT stack(2, 1, 2, 3) | struct |
-| org.apache.spark.sql.catalyst.expressions.StartsWithExpressionBuilder | startswith | SELECT startswith('Spark SQL', 'Spark') | struct |
-| org.apache.spark.sql.catalyst.expressions.StringInstr | instr | SELECT instr('SparkSQL', 'SQL') | struct |
-| org.apache.spark.sql.catalyst.expressions.StringLocate | locate | SELECT locate('bar', 'foobarbar') | struct |
-| org.apache.spark.sql.catalyst.expressions.StringLocate | position | SELECT position('bar', 'foobarbar') | struct |
-| org.apache.spark.sql.catalyst.expressions.StringRepeat | repeat | SELECT repeat('123', 2) | struct |
-| org.apache.spark.sql.catalyst.expressions.StringReplace | replace | SELECT replace('ABCabc', 'abc', 'DEF') | struct |
-| org.apache.spark.sql.catalyst.expressions.StringSpace | space | SELECT concat(space(2), '1') | struct |
-| org.apache.spark.sql.catalyst.expressions.StringSplit | split | SELECT split('oneAtwoBthreeC', '[ABC]') | struct> |
-| org.apache.spark.sql.catalyst.expressions.StringToMap | str_to_map | SELECT str_to_map('a:1,b:2,c:3', ',', ':') | struct> |
-| org.apache.spark.sql.catalyst.expressions.StringTranslate | translate | SELECT translate('AaBbCc', 'abc', '123') | struct |
-| org.apache.spark.sql.catalyst.expressions.StringTrim | trim | SELECT trim(' SparkSQL ') | struct |
-| org.apache.spark.sql.catalyst.expressions.StringTrimBoth | btrim | SELECT btrim(' SparkSQL ') | struct |
-| org.apache.spark.sql.catalyst.expressions.StringTrimLeft | ltrim | SELECT ltrim(' SparkSQL ') | struct |
-| org.apache.spark.sql.catalyst.expressions.StringTrimRight | rtrim | SELECT rtrim(' SparkSQL ') | struct |
-| org.apache.spark.sql.catalyst.expressions.StructsToCsv | to_csv | SELECT to_csv(named_struct('a', 1, 'b', 2)) | struct |
-| org.apache.spark.sql.catalyst.expressions.StructsToJson | to_json | SELECT to_json(named_struct('a', 1, 'b', 2)) | struct |
-| org.apache.spark.sql.catalyst.expressions.StructsToXml | to_xml | SELECT to_xml(named_struct('a', 1, 'b', 2)) | struct |
-| org.apache.spark.sql.catalyst.expressions.Substring | substr | SELECT substr('Spark SQL', 5) | struct |
-| org.apache.spark.sql.catalyst.expressions.Substring | substring | SELECT substring('Spark SQL', 5) | struct |
-| org.apache.spark.sql.catalyst.expressions.SubstringIndex | substring_index | SELECT substring_index('www.apache.org', '.', 2) | struct |
-| org.apache.spark.sql.catalyst.expressions.Subtract | - | SELECT 2 - 1 | struct<(2 - 1):int> |
-| org.apache.spark.sql.catalyst.expressions.Tan | tan | SELECT tan(0) | struct |
-| org.apache.spark.sql.catalyst.expressions.Tanh | tanh | SELECT tanh(0) | struct |
-| org.apache.spark.sql.catalyst.expressions.TimeWindow | window | SELECT a, window.start, window.end, count(*) as cnt FROM VALUES ('A1', '2021-01-01 00:00:00'), ('A1', '2021-01-01 00:04:30'), ('A1', '2021-01-01 00:06:00'), ('A2', '2021-01-01 00:01:00') AS tab(a, b) GROUP by a, window(b, '5 minutes') ORDER BY a, start | struct |
-| org.apache.spark.sql.catalyst.expressions.ToBinary | to_binary | SELECT to_binary('abc', 'utf-8') | struct |
-| org.apache.spark.sql.catalyst.expressions.ToCharacterBuilder | to_char | SELECT to_char(454, '999') | struct |
-| org.apache.spark.sql.catalyst.expressions.ToCharacterBuilder | to_varchar | SELECT to_varchar(454, '999') | struct |
-| org.apache.spark.sql.catalyst.expressions.ToDegrees | degrees | SELECT degrees(3.141592653589793) | struct |
-| org.apache.spark.sql.catalyst.expressions.ToNumber | to_number | SELECT to_number('454', '999') | struct |
-| org.apache.spark.sql.catalyst.expressions.ToRadians | radians | SELECT radians(180) | struct |
-| org.apache.spark.sql.catalyst.expressions.ToUTCTimestamp | to_utc_timestamp | SELECT to_utc_timestamp('2016-08-31', 'Asia/Seoul') | struct |
-| org.apache.spark.sql.catalyst.expressions.ToUnixTimestamp | to_unix_timestamp | SELECT to_unix_timestamp('2016-04-08', 'yyyy-MM-dd') | struct |
-| org.apache.spark.sql.catalyst.expressions.TransformKeys | transform_keys | SELECT transform_keys(map_from_arrays(array(1, 2, 3), array(1, 2, 3)), (k, v) -> k + 1) | struct> |
-| org.apache.spark.sql.catalyst.expressions.TransformValues | transform_values | SELECT transform_values(map_from_arrays(array(1, 2, 3), array(1, 2, 3)), (k, v) -> v + 1) | struct> |
-| org.apache.spark.sql.catalyst.expressions.TruncDate | trunc | SELECT trunc('2019-08-04', 'week') | struct |
-| org.apache.spark.sql.catalyst.expressions.TruncTimestamp | date_trunc | SELECT date_trunc('YEAR', '2015-03-05T09:32:05.359') | struct |
-| org.apache.spark.sql.catalyst.expressions.TryAdd | try_add | SELECT try_add(1, 2) | struct |
-| org.apache.spark.sql.catalyst.expressions.TryAesDecrypt | try_aes_decrypt | SELECT try_aes_decrypt(unhex('6E7CA17BBB468D3084B5744BCA729FB7B2B7BCB8E4472847D02670489D95FA97DBBA7D3210'), '0000111122223333', 'GCM') | struct |
-| org.apache.spark.sql.catalyst.expressions.TryDivide | try_divide | SELECT try_divide(3, 2) | struct |
-| org.apache.spark.sql.catalyst.expressions.TryElementAt | try_element_at | SELECT try_element_at(array(1, 2, 3), 2) | struct |
-| org.apache.spark.sql.catalyst.expressions.TryMultiply | try_multiply | SELECT try_multiply(2, 3) | struct |
-| org.apache.spark.sql.catalyst.expressions.TryReflect | try_reflect | SELECT try_reflect('java.util.UUID', 'randomUUID') | struct |
-| org.apache.spark.sql.catalyst.expressions.TrySubtract | try_subtract | SELECT try_subtract(2, 1) | struct |
-| org.apache.spark.sql.catalyst.expressions.TryToBinary | try_to_binary | SELECT try_to_binary('abc', 'utf-8') | struct |
-| org.apache.spark.sql.catalyst.expressions.TryToNumber | try_to_number | SELECT try_to_number('454', '999') | struct |
-| org.apache.spark.sql.catalyst.expressions.TryToTimestampExpressionBuilder | try_to_timestamp | SELECT try_to_timestamp('2016-12-31 00:12:00') | struct |
-| org.apache.spark.sql.catalyst.expressions.TypeOf | typeof | SELECT typeof(1) | struct |
-| org.apache.spark.sql.catalyst.expressions.UnBase64 | unbase64 | SELECT unbase64('U3BhcmsgU1FM') | struct |
-| org.apache.spark.sql.catalyst.expressions.UnaryMinus | negative | SELECT negative(1) | struct |
-| org.apache.spark.sql.catalyst.expressions.UnaryPositive | positive | SELECT positive(1) | struct<(+ 1):int> |
-| org.apache.spark.sql.catalyst.expressions.Unhex | unhex | SELECT decode(unhex('537061726B2053514C'), 'UTF-8') | struct |
-| org.apache.spark.sql.catalyst.expressions.UnixDate | unix_date | SELECT unix_date(DATE("1970-01-02")) | struct |
-| org.apache.spark.sql.catalyst.expressions.UnixMicros | unix_micros | SELECT unix_micros(TIMESTAMP('1970-01-01 00:00:01Z')) | struct |
-| org.apache.spark.sql.catalyst.expressions.UnixMillis | unix_millis | SELECT unix_millis(TIMESTAMP('1970-01-01 00:00:01Z')) | struct |
-| org.apache.spark.sql.catalyst.expressions.UnixSeconds | unix_seconds | SELECT unix_seconds(TIMESTAMP('1970-01-01 00:00:01Z')) | struct |
-| org.apache.spark.sql.catalyst.expressions.UnixTimestamp | unix_timestamp | SELECT unix_timestamp() | struct |
-| org.apache.spark.sql.catalyst.expressions.Upper | ucase | SELECT ucase('SparkSql') | struct |
-| org.apache.spark.sql.catalyst.expressions.Upper | upper | SELECT upper('SparkSql') | struct |
-| org.apache.spark.sql.catalyst.expressions.UrlDecode | url_decode | SELECT url_decode('https%3A%2F%2Fspark.apache.org') | struct |
-| org.apache.spark.sql.catalyst.expressions.UrlEncode | url_encode | SELECT url_encode('https://spark.apache.org') | struct |
-| org.apache.spark.sql.catalyst.expressions.Uuid | uuid | SELECT uuid() | struct |
-| org.apache.spark.sql.catalyst.expressions.WeekDay | weekday | SELECT weekday('2009-07-30') | struct |
-| org.apache.spark.sql.catalyst.expressions.WeekOfYear | weekofyear | SELECT weekofyear('2008-02-20') | struct |
-| org.apache.spark.sql.catalyst.expressions.WidthBucket | width_bucket | SELECT width_bucket(5.3, 0.2, 10.6, 5) | struct |
-| org.apache.spark.sql.catalyst.expressions.WindowTime | window_time | SELECT a, window.start as start, window.end as end, window_time(window), cnt FROM (SELECT a, window, count(*) as cnt FROM VALUES ('A1', '2021-01-01 00:00:00'), ('A1', '2021-01-01 00:04:30'), ('A1', '2021-01-01 00:06:00'), ('A2', '2021-01-01 00:01:00') AS tab(a, b) GROUP by a, window(b, '5 minutes') ORDER BY a, window.start) | struct |
-| org.apache.spark.sql.catalyst.expressions.XmlToStructs | from_xml | SELECT from_xml('10.8
', 'a INT, b DOUBLE') | struct10.8):struct> |
-| org.apache.spark.sql.catalyst.expressions.XxHash64 | xxhash64 | SELECT xxhash64('Spark', array(123), 2) | struct |
-| org.apache.spark.sql.catalyst.expressions.Year | year | SELECT year('2016-07-30') | struct |
-| org.apache.spark.sql.catalyst.expressions.ZipWith | zip_with | SELECT zip_with(array(1, 2, 3), array('a', 'b', 'c'), (x, y) -> (y, x)) | struct>> |
-| org.apache.spark.sql.catalyst.expressions.aggregate.AnyValue | any_value | SELECT any_value(col) FROM VALUES (10), (5), (20) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile | approx_percentile | SELECT approx_percentile(col, array(0.5, 0.4, 0.1), 100) FROM VALUES (0), (1), (2), (10) AS tab(col) | struct> |
-| org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile | percentile_approx | SELECT percentile_approx(col, array(0.5, 0.4, 0.1), 100) FROM VALUES (0), (1), (2), (10) AS tab(col) | struct> |
-| org.apache.spark.sql.catalyst.expressions.aggregate.Average | avg | SELECT avg(col) FROM VALUES (1), (2), (3) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.Average | mean | SELECT mean(col) FROM VALUES (1), (2), (3) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.BitAndAgg | bit_and | SELECT bit_and(col) FROM VALUES (3), (5) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.BitOrAgg | bit_or | SELECT bit_or(col) FROM VALUES (3), (5) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.BitXorAgg | bit_xor | SELECT bit_xor(col) FROM VALUES (3), (5) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.BoolAnd | bool_and | SELECT bool_and(col) FROM VALUES (true), (true), (true) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.BoolAnd | every | SELECT every(col) FROM VALUES (true), (true), (true) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.BoolOr | any | SELECT any(col) FROM VALUES (true), (false), (false) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.BoolOr | bool_or | SELECT bool_or(col) FROM VALUES (true), (false), (false) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.BoolOr | some | SELECT some(col) FROM VALUES (true), (false), (false) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.CollectList | array_agg | SELECT array_agg(col) FROM VALUES (1), (2), (1) AS tab(col) | struct> |
-| org.apache.spark.sql.catalyst.expressions.aggregate.CollectList | collect_list | SELECT collect_list(col) FROM VALUES (1), (2), (1) AS tab(col) | struct> |
-| org.apache.spark.sql.catalyst.expressions.aggregate.CollectSet | collect_set | SELECT collect_set(col) FROM VALUES (1), (2), (1) AS tab(col) | struct> |
-| org.apache.spark.sql.catalyst.expressions.aggregate.Corr | corr | SELECT corr(c1, c2) FROM VALUES (3, 2), (3, 3), (6, 4) as tab(c1, c2) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.Count | count | SELECT count(*) FROM VALUES (NULL), (5), (5), (20) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.CountIf | count_if | SELECT count_if(col % 2 = 0) FROM VALUES (NULL), (0), (1), (2), (3) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.CountMinSketchAggExpressionBuilder | count_min_sketch | SELECT hex(count_min_sketch(col, 0.5d, 0.5d, 1)) FROM VALUES (1), (2), (1) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.CovPopulation | covar_pop | SELECT covar_pop(c1, c2) FROM VALUES (1,1), (2,2), (3,3) AS tab(c1, c2) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.CovSample | covar_samp | SELECT covar_samp(c1, c2) FROM VALUES (1,1), (2,2), (3,3) AS tab(c1, c2) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.First | first | SELECT first(col) FROM VALUES (10), (5), (20) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.First | first_value | SELECT first_value(col) FROM VALUES (10), (5), (20) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.HistogramNumeric | histogram_numeric | SELECT histogram_numeric(col, 5) FROM VALUES (0), (1), (2), (10) AS tab(col) | struct>> |
-| org.apache.spark.sql.catalyst.expressions.aggregate.HllSketchAgg | hll_sketch_agg | SELECT hll_sketch_estimate(hll_sketch_agg(col, 12)) FROM VALUES (1), (1), (2), (2), (3) tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.HllUnionAgg | hll_union_agg | SELECT hll_sketch_estimate(hll_union_agg(sketch, true)) FROM (SELECT hll_sketch_agg(col) as sketch FROM VALUES (1) tab(col) UNION ALL SELECT hll_sketch_agg(col, 20) as sketch FROM VALUES (1) tab(col)) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.HyperLogLogPlusPlus | approx_count_distinct | SELECT approx_count_distinct(col1) FROM VALUES (1), (1), (2), (2), (3) tab(col1) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.Kurtosis | kurtosis | SELECT kurtosis(col) FROM VALUES (-10), (-20), (100), (1000) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.Last | last | SELECT last(col) FROM VALUES (10), (5), (20) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.Last | last_value | SELECT last_value(col) FROM VALUES (10), (5), (20) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.Max | max | SELECT max(col) FROM VALUES (10), (50), (20) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.MaxBy | max_by | SELECT max_by(x, y) FROM VALUES ('a', 10), ('b', 50), ('c', 20) AS tab(x, y) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.Median | median | SELECT median(col) FROM VALUES (0), (10) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.Min | min | SELECT min(col) FROM VALUES (10), (-1), (20) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.MinBy | min_by | SELECT min_by(x, y) FROM VALUES ('a', 10), ('b', 50), ('c', 20) AS tab(x, y) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.ModeBuilder | mode | SELECT mode(col) FROM VALUES (0), (10), (10) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.Percentile | percentile | SELECT percentile(col, 0.3) FROM VALUES (0), (10) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.PercentileContBuilder | percentile_cont | SELECT percentile_cont(0.25) WITHIN GROUP (ORDER BY col) FROM VALUES (0), (10) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.PercentileDiscBuilder | percentile_disc | SELECT percentile_disc(0.25) WITHIN GROUP (ORDER BY col) FROM VALUES (0), (10) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.RegrAvgX | regr_avgx | SELECT regr_avgx(y, x) FROM VALUES (1, 2), (2, 2), (2, 3), (2, 4) AS tab(y, x) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.RegrAvgY | regr_avgy | SELECT regr_avgy(y, x) FROM VALUES (1, 2), (2, 2), (2, 3), (2, 4) AS tab(y, x) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.RegrCount | regr_count | SELECT regr_count(y, x) FROM VALUES (1, 2), (2, 2), (2, 3), (2, 4) AS tab(y, x) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.RegrIntercept | regr_intercept | SELECT regr_intercept(y, x) FROM VALUES (1,1), (2,2), (3,3) AS tab(y, x) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.RegrR2 | regr_r2 | SELECT regr_r2(y, x) FROM VALUES (1, 2), (2, 2), (2, 3), (2, 4) AS tab(y, x) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.RegrSXX | regr_sxx | SELECT regr_sxx(y, x) FROM VALUES (1, 2), (2, 2), (2, 3), (2, 4) AS tab(y, x) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.RegrSXY | regr_sxy | SELECT regr_sxy(y, x) FROM VALUES (1, 2), (2, 2), (2, 3), (2, 4) AS tab(y, x) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.RegrSYY | regr_syy | SELECT regr_syy(y, x) FROM VALUES (1, 2), (2, 2), (2, 3), (2, 4) AS tab(y, x) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.RegrSlope | regr_slope | SELECT regr_slope(y, x) FROM VALUES (1,1), (2,2), (3,3) AS tab(y, x) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.Skewness | skewness | SELECT skewness(col) FROM VALUES (-10), (-20), (100), (1000) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.StddevPop | stddev_pop | SELECT stddev_pop(col) FROM VALUES (1), (2), (3) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.StddevSamp | std | SELECT std(col) FROM VALUES (1), (2), (3) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.StddevSamp | stddev | SELECT stddev(col) FROM VALUES (1), (2), (3) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.StddevSamp | stddev_samp | SELECT stddev_samp(col) FROM VALUES (1), (2), (3) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.Sum | sum | SELECT sum(col) FROM VALUES (5), (10), (15) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.TryAverageExpressionBuilder | try_avg | SELECT try_avg(col) FROM VALUES (1), (2), (3) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.TrySumExpressionBuilder | try_sum | SELECT try_sum(col) FROM VALUES (5), (10), (15) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.VariancePop | var_pop | SELECT var_pop(col) FROM VALUES (1), (2), (3) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.VarianceSamp | var_samp | SELECT var_samp(col) FROM VALUES (1), (2), (3) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.aggregate.VarianceSamp | variance | SELECT variance(col) FROM VALUES (1), (2), (3) AS tab(col) | struct |
-| org.apache.spark.sql.catalyst.expressions.variant.ParseJson | parse_json | SELECT parse_json('{"a":1,"b":0.8}') | struct |
-| org.apache.spark.sql.catalyst.expressions.xml.XPathBoolean | xpath_boolean | SELECT xpath_boolean('1','a/b') | struct1, a/b):boolean> |
-| org.apache.spark.sql.catalyst.expressions.xml.XPathDouble | xpath_double | SELECT xpath_double('12', 'sum(a/b)') | struct12, sum(a/b)):double> |
-| org.apache.spark.sql.catalyst.expressions.xml.XPathDouble | xpath_number | SELECT xpath_number('12', 'sum(a/b)') | struct12, sum(a/b)):double> |
-| org.apache.spark.sql.catalyst.expressions.xml.XPathFloat | xpath_float | SELECT xpath_float('12', 'sum(a/b)') | struct12, sum(a/b)):float> |
-| org.apache.spark.sql.catalyst.expressions.xml.XPathInt | xpath_int | SELECT xpath_int('12', 'sum(a/b)') | struct12, sum(a/b)):int> |
-| org.apache.spark.sql.catalyst.expressions.xml.XPathList | xpath | SELECT xpath('b1b2b3c1c2','a/b/text()') | structb1b2b3c1c2, a/b/text()):array> |
-| org.apache.spark.sql.catalyst.expressions.xml.XPathLong | xpath_long | SELECT xpath_long('12', 'sum(a/b)') | struct12, sum(a/b)):bigint> |
-| org.apache.spark.sql.catalyst.expressions.xml.XPathShort | xpath_short | SELECT xpath_short('12', 'sum(a/b)') | struct12, sum(a/b)):smallint> |
-| org.apache.spark.sql.catalyst.expressions.xml.XPathString | xpath_string | SELECT xpath_string('bcc','a/c') | structbcc, a/c):string> |
\ No newline at end of file
+| Class name | Function name or alias | Query example | Output schema |
+| ---------- |-----------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| org.apache.spark.sql.catalyst.expressions.Abs | abs | SELECT abs(-1) | struct |
+| org.apache.spark.sql.catalyst.expressions.Acos | acos | SELECT acos(1) | struct |
+| org.apache.spark.sql.catalyst.expressions.Acosh | acosh | SELECT acosh(1) | struct |
+| org.apache.spark.sql.catalyst.expressions.Add | + | SELECT 1 + 2 | struct<(1 + 2):int> |
+| org.apache.spark.sql.catalyst.expressions.AddMonths | add_months | SELECT add_months('2016-08-31', 1) | struct |
+| org.apache.spark.sql.catalyst.expressions.AesDecrypt | aes_decrypt | SELECT aes_decrypt(unhex('83F16B2AA704794132802D248E6BFD4E380078182D1544813898AC97E709B28A94'), '0000111122223333') | struct |
+| org.apache.spark.sql.catalyst.expressions.AesEncrypt | aes_encrypt | SELECT hex(aes_encrypt('Spark', '0000111122223333')) | struct |
+| org.apache.spark.sql.catalyst.expressions.And | and | SELECT true and true | struct<(true AND true):boolean> |
+| org.apache.spark.sql.catalyst.expressions.ArrayAggregate | aggregate | SELECT aggregate(array(1, 2, 3), 0, (acc, x) -> acc + x) | struct |
+| org.apache.spark.sql.catalyst.expressions.ArrayAggregate | reduce | SELECT reduce(array(1, 2, 3), 0, (acc, x) -> acc + x) | struct |
+| org.apache.spark.sql.catalyst.expressions.ArrayAppend | array_append | SELECT array_append(array('b', 'd', 'c', 'a'), 'd') | struct> |
+| org.apache.spark.sql.catalyst.expressions.ArrayCompact | array_compact | SELECT array_compact(array(1, 2, 3, null)) | struct> |
+| org.apache.spark.sql.catalyst.expressions.ArrayContains | array_contains | SELECT array_contains(array(1, 2, 3), 2) | struct |
+| org.apache.spark.sql.catalyst.expressions.ArrayDistinct | array_distinct | SELECT array_distinct(array(1, 2, 3, null, 3)) | struct> |
+| org.apache.spark.sql.catalyst.expressions.ArrayExcept | array_except | SELECT array_except(array(1, 2, 3), array(1, 3, 5)) | struct> |
+| org.apache.spark.sql.catalyst.expressions.ArrayExists | exists | SELECT exists(array(1, 2, 3), x -> x % 2 == 0) | struct |
+| org.apache.spark.sql.catalyst.expressions.ArrayFilter | filter | SELECT filter(array(1, 2, 3), x -> x % 2 == 1) | struct> |
+| org.apache.spark.sql.catalyst.expressions.ArrayForAll | forall | SELECT forall(array(1, 2, 3), x -> x % 2 == 0) | struct |
+| org.apache.spark.sql.catalyst.expressions.ArrayInsert | array_insert | SELECT array_insert(array(1, 2, 3, 4), 5, 5) | struct> |
+| org.apache.spark.sql.catalyst.expressions.ArrayIntersect | array_intersect | SELECT array_intersect(array(1, 2, 3), array(1, 3, 5)) | struct> |
+| org.apache.spark.sql.catalyst.expressions.ArrayJoin | array_join | SELECT array_join(array('hello', 'world'), ' ') | struct |
+| org.apache.spark.sql.catalyst.expressions.ArrayMax | array_max | SELECT array_max(array(1, 20, null, 3)) | struct |
+| org.apache.spark.sql.catalyst.expressions.ArrayMin | array_min | SELECT array_min(array(1, 20, null, 3)) | struct |
+| org.apache.spark.sql.catalyst.expressions.ArrayPosition | array_position | SELECT array_position(array(312, 773, 708, 708), 708) | struct |
+| org.apache.spark.sql.catalyst.expressions.ArrayPrepend | array_prepend | SELECT array_prepend(array('b', 'd', 'c', 'a'), 'd') | struct> |
+| org.apache.spark.sql.catalyst.expressions.ArrayRemove | array_remove | SELECT array_remove(array(1, 2, 3, null, 3), 3) | struct> |
+| org.apache.spark.sql.catalyst.expressions.ArrayRepeat | array_repeat | SELECT array_repeat('123', 2) | struct> |
+| org.apache.spark.sql.catalyst.expressions.ArraySize | array_size | SELECT array_size(array('b', 'd', 'c', 'a')) | struct |
+| org.apache.spark.sql.catalyst.expressions.ArraySort | array_sort | SELECT array_sort(array(5, 6, 1), (left, right) -> case when left < right then -1 when left > right then 1 else 0 end) | struct namedlambdavariable()) THEN 1 ELSE 0 END, namedlambdavariable(), namedlambdavariable())):array> |
+| org.apache.spark.sql.catalyst.expressions.ArrayTransform | transform | SELECT transform(array(1, 2, 3), x -> x + 1) | struct> |
+| org.apache.spark.sql.catalyst.expressions.ArrayUnion | array_union | SELECT array_union(array(1, 2, 3), array(1, 3, 5)) | struct> |
+| org.apache.spark.sql.catalyst.expressions.ArraysOverlap | arrays_overlap | SELECT arrays_overlap(array(1, 2, 3), array(3, 4, 5)) | struct |
+| org.apache.spark.sql.catalyst.expressions.ArraysZip | arrays_zip | SELECT arrays_zip(array(1, 2, 3), array(2, 3, 4)) | struct>> |
+| org.apache.spark.sql.catalyst.expressions.Ascii | ascii | SELECT ascii('222') | struct |
+| org.apache.spark.sql.catalyst.expressions.Asin | asin | SELECT asin(0) | struct |
+| org.apache.spark.sql.catalyst.expressions.Asinh | asinh | SELECT asinh(0) | struct |
+| org.apache.spark.sql.catalyst.expressions.AssertTrue | assert_true | SELECT assert_true(0 < 1) | struct |
+| org.apache.spark.sql.catalyst.expressions.Atan | atan | SELECT atan(0) | struct |
+| org.apache.spark.sql.catalyst.expressions.Atan2 | atan2 | SELECT atan2(0, 0) | struct |
+| org.apache.spark.sql.catalyst.expressions.Atanh | atanh | SELECT atanh(0) | struct |
+| org.apache.spark.sql.catalyst.expressions.BRound | bround | SELECT bround(2.5, 0) | struct |
+| org.apache.spark.sql.catalyst.expressions.Base64 | base64 | SELECT base64('Spark SQL') | struct |
+| org.apache.spark.sql.catalyst.expressions.Between | between | SELECT 0.5 between 0.1 AND 1.0 | struct |
+| org.apache.spark.sql.catalyst.expressions.Bin | bin | SELECT bin(13) | struct |
+| org.apache.spark.sql.catalyst.expressions.BitLength | bit_length | SELECT bit_length('Spark SQL') | struct |
+| org.apache.spark.sql.catalyst.expressions.BitmapBitPosition | bitmap_bit_position | SELECT bitmap_bit_position(1) | struct |
+| org.apache.spark.sql.catalyst.expressions.BitmapBucketNumber | bitmap_bucket_number | SELECT bitmap_bucket_number(123) | struct |
+| org.apache.spark.sql.catalyst.expressions.BitmapConstructAgg | bitmap_construct_agg | SELECT substring(hex(bitmap_construct_agg(bitmap_bit_position(col))), 0, 6) FROM VALUES (1), (2), (3) AS tab(col) | struct |
+| org.apache.spark.sql.catalyst.expressions.BitmapCount | bitmap_count | SELECT bitmap_count(X '1010') | struct |
+| org.apache.spark.sql.catalyst.expressions.BitmapOrAgg | bitmap_or_agg | SELECT substring(hex(bitmap_or_agg(col)), 0, 6) FROM VALUES (X '10'), (X '20'), (X '40') AS tab(col) | struct |
+| org.apache.spark.sql.catalyst.expressions.BitwiseAnd | & | SELECT 3 & 5 | struct<(3 & 5):int> |
+| org.apache.spark.sql.catalyst.expressions.BitwiseCount | bit_count | SELECT bit_count(0) | struct |
+| org.apache.spark.sql.catalyst.expressions.BitwiseGet | bit_get | SELECT bit_get(11, 0) | struct |
+| org.apache.spark.sql.catalyst.expressions.BitwiseGet | getbit | SELECT getbit(11, 0) | struct |
+| org.apache.spark.sql.catalyst.expressions.BitwiseNot | ~ | SELECT ~ 0 | struct<~0:int> |
+| org.apache.spark.sql.catalyst.expressions.BitwiseOr | | | SELECT 3 | 5 | struct<(3 | 5):int> |
+| org.apache.spark.sql.catalyst.expressions.BitwiseXor | ^ | SELECT 3 ^ 5 | struct<(3 ^ 5):int> |
+| org.apache.spark.sql.catalyst.expressions.CallMethodViaReflection | java_method | SELECT java_method('java.util.UUID', 'randomUUID') | struct |
+| org.apache.spark.sql.catalyst.expressions.CallMethodViaReflection | reflect | SELECT reflect('java.util.UUID', 'randomUUID') | struct |
+| org.apache.spark.sql.catalyst.expressions.CaseWhen | when | SELECT CASE WHEN 1 > 0 THEN 1 WHEN 2 > 0 THEN 2.0 ELSE 1.2 END | struct 0) THEN 1 WHEN (2 > 0) THEN 2.0 ELSE 1.2 END:decimal(11,1)> |
+| org.apache.spark.sql.catalyst.expressions.Cast | bigint | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.Cast | binary | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.Cast | boolean | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.Cast | cast | SELECT cast('10' as int) | struct |
+| org.apache.spark.sql.catalyst.expressions.Cast | date | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.Cast | decimal | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.Cast | double | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.Cast | float | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.Cast | int | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.Cast | smallint | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.Cast | string | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.Cast | timestamp | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.Cast | tinyint | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.Cbrt | cbrt | SELECT cbrt(27.0) | struct |
+| org.apache.spark.sql.catalyst.expressions.CeilExpressionBuilder | ceil | SELECT ceil(-0.1) | struct |
+| org.apache.spark.sql.catalyst.expressions.CeilExpressionBuilder | ceiling | SELECT ceiling(-0.1) | struct |
+| org.apache.spark.sql.catalyst.expressions.Chr | char | SELECT char(65) | struct |
+| org.apache.spark.sql.catalyst.expressions.Chr | chr | SELECT chr(65) | struct |
+| org.apache.spark.sql.catalyst.expressions.Coalesce | coalesce | SELECT coalesce(NULL, 1, NULL) | struct |
+| org.apache.spark.sql.catalyst.expressions.Concat | concat | SELECT concat('Spark', 'SQL') | struct |
+| org.apache.spark.sql.catalyst.expressions.ConcatWs | concat_ws | SELECT concat_ws(' ', 'Spark', 'SQL') | struct |
+| org.apache.spark.sql.catalyst.expressions.ContainsExpressionBuilder | contains | SELECT contains('Spark SQL', 'Spark') | struct |
+| org.apache.spark.sql.catalyst.expressions.Conv | conv | SELECT conv('100', 2, 10) | struct |
+| org.apache.spark.sql.catalyst.expressions.ConvertTimezone | convert_timezone | SELECT convert_timezone('Europe/Brussels', 'America/Los_Angeles', timestamp_ntz'2021-12-06 00:00:00') | struct |
+| org.apache.spark.sql.catalyst.expressions.Cos | cos | SELECT cos(0) | struct |
+| org.apache.spark.sql.catalyst.expressions.Cosh | cosh | SELECT cosh(0) | struct |
+| org.apache.spark.sql.catalyst.expressions.Cot | cot | SELECT cot(1) | struct |
+| org.apache.spark.sql.catalyst.expressions.Crc32 | crc32 | SELECT crc32('Spark') | struct |
+| org.apache.spark.sql.catalyst.expressions.CreateArray | array | SELECT array(1, 2, 3) | struct> |
+| org.apache.spark.sql.catalyst.expressions.CreateMap | map | SELECT map(1.0, '2', 3.0, '4') | struct