From 6393afada79ab4d6f7a45139017b21021ecfaec1 Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Tue, 17 Sep 2024 14:47:20 +0200 Subject: [PATCH] [SPARK-49505][SQL] Create new SQL functions "randstr" and "uniform" to generate random strings or numbers within ranges ### What changes were proposed in this pull request? This PR introduces two new SQL functions "randstr" and "uniform" to generate random strings or numbers within ranges. * The "randstr" function returns a string of the specified length whose characters are chosen uniformly at random from the following pool of characters: 0-9, a-z, A-Z. The random seed is optional. The string length must be a constant two-byte or four-byte integer (SMALLINT or INT, respectively). * The "uniform" function returns a random value with independent and identically distributed values with the specified range of numbers. The random seed is optional. The provided numbers specifying the minimum and maximum values of the range must be constant. If both of these numbers are integers, then the result will also be an integer. Otherwise if one or both of these are floating-point numbers, then the result will also be a floating-point number. For example: ``` SELECT randstr(5); > ceV0P SELECT randstr(10, 0) FROM VALUES (0), (1), (2) tab(col); > ceV0PXaR2I fYxVfArnv7 iSIv0VT2XL SELECT uniform(10, 20.0F); > 17.604954 SELECT uniform(10, 20, 0) FROM VALUES (0), (1), (2) tab(col); > 15 16 17 ``` ### Why are the changes needed? This improves the SQL functionality of Apache Spark and improves its parity with other systems: * https://clickhouse.com/docs/en/sql-reference/functions/random-functions#randuniform * https://docs.snowflake.com/en/sql-reference/functions/uniform * https://www.microfocus.com/documentation/silk-test/21.0.2/en/silktestclassic-help-en/STCLASSIC-8BFE8661-RANDSTRFUNCTION-REF.html * https://docs.snowflake.com/en/sql-reference/functions/randstr ### Does this PR introduce _any_ user-facing change? Yes, see above. ### How was this patch tested? This PR adds golden file based test coverage. ### Was this patch authored or co-authored using generative AI tooling? Not this time. Closes #48004 from dtenedor/uniform-randstr-functions. Authored-by: Daniel Tenedorio Signed-off-by: Max Gekk --- .../catalyst/analysis/FunctionRegistry.scala | 2 + .../expressions/randomExpressions.scala | 264 +++++++++- .../catalyst/expressions/RandomSuite.scala | 24 + .../sql-functions/sql-expression-schema.md | 2 + .../sql-tests/analyzer-results/random.sql.out | 401 +++++++++++++++ .../resources/sql-tests/inputs/random.sql | 42 +- .../sql-tests/results/random.sql.out | 469 ++++++++++++++++++ 7 files changed, 1191 insertions(+), 13 deletions(-) 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 5a3c4b0ec8696..d03d8114e9976 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 @@ -384,7 +384,9 @@ object FunctionRegistry { expression[Rand]("rand"), expression[Rand]("random", true, Some("3.0.0")), expression[Randn]("randn"), + expression[RandStr]("randstr"), expression[Stack]("stack"), + expression[Uniform]("uniform"), expression[ZeroIfNull]("zeroifnull"), CaseWhen.registryEntry, 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 f5db972a28643..ea9ca451c2cb1 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 @@ -17,13 +17,18 @@ package org.apache.spark.sql.catalyst.expressions +import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.UnresolvedSeed +import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, UnresolvedSeed} +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch +import org.apache.spark.sql.catalyst.expressions.ExpectsInputTypes.{ordinalNumber, toSQLExpr, toSQLType} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode, FalseLiteral} import org.apache.spark.sql.catalyst.expressions.codegen.Block._ -import org.apache.spark.sql.catalyst.trees.TreePattern.{EXPRESSION_WITH_RANDOM_SEED, TreePattern} +import org.apache.spark.sql.catalyst.trees.{BinaryLike, TernaryLike, UnaryLike} +import org.apache.spark.sql.catalyst.trees.TreePattern.{EXPRESSION_WITH_RANDOM_SEED, RUNTIME_REPLACEABLE, TreePattern} import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.random.XORShiftRandom /** @@ -33,8 +38,7 @@ import org.apache.spark.util.random.XORShiftRandom * * Since this expression is stateful, it cannot be a case object. */ -abstract class RDG extends UnaryExpression with ExpectsInputTypes with Nondeterministic - with ExpressionWithRandomSeed { +trait RDG extends Expression with ExpressionWithRandomSeed { /** * Record ID within each partition. By being transient, the Random Number Generator is * reset every time we serialize and deserialize and initialize it. @@ -43,12 +47,6 @@ abstract class RDG extends UnaryExpression with ExpectsInputTypes with Nondeterm override def stateful: Boolean = true - override protected def initializeInternal(partitionIndex: Int): Unit = { - rng = new XORShiftRandom(seed + partitionIndex) - } - - override def seedExpression: Expression = child - @transient protected lazy val seed: Long = seedExpression match { case e if e.dataType == IntegerType => e.eval().asInstanceOf[Int] case e if e.dataType == LongType => e.eval().asInstanceOf[Long] @@ -57,6 +55,15 @@ abstract class RDG extends UnaryExpression with ExpectsInputTypes with Nondeterm override def nullable: Boolean = false override def dataType: DataType = DoubleType +} + +abstract class NondeterministicUnaryRDG + extends RDG with UnaryLike[Expression] with Nondeterministic with ExpectsInputTypes { + override def seedExpression: Expression = child + + override protected def initializeInternal(partitionIndex: Int): Unit = { + rng = new XORShiftRandom(seed + partitionIndex) + } override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(IntegerType, LongType)) } @@ -99,7 +106,7 @@ private[catalyst] object ExpressionWithRandomSeed { since = "1.5.0", group = "math_funcs") // scalastyle:on line.size.limit -case class Rand(child: Expression, hideSeed: Boolean = false) extends RDG { +case class Rand(child: Expression, hideSeed: Boolean = false) extends NondeterministicUnaryRDG { def this() = this(UnresolvedSeed, true) @@ -150,7 +157,7 @@ object Rand { since = "1.5.0", group = "math_funcs") // scalastyle:on line.size.limit -case class Randn(child: Expression, hideSeed: Boolean = false) extends RDG { +case class Randn(child: Expression, hideSeed: Boolean = false) extends NondeterministicUnaryRDG { def this() = this(UnresolvedSeed, true) @@ -181,3 +188,236 @@ case class Randn(child: Expression, hideSeed: Boolean = false) extends RDG { object Randn { def apply(seed: Long): Randn = Randn(Literal(seed, LongType)) } + +@ExpressionDescription( + usage = """ + _FUNC_(min, max[, seed]) - Returns a random value with independent and identically + distributed (i.i.d.) values with the specified range of numbers. The random seed is optional. + The provided numbers specifying the minimum and maximum values of the range must be constant. + If both of these numbers are integers, then the result will also be an integer. Otherwise if + one or both of these are floating-point numbers, then the result will also be a floating-point + number. + """, + examples = """ + Examples: + > SELECT _FUNC_(10, 20, 0) > 0 AS result; + true + """, + since = "4.0.0", + group = "math_funcs") +case class Uniform(min: Expression, max: Expression, seedExpression: Expression) + extends RuntimeReplaceable with TernaryLike[Expression] with RDG { + def this(min: Expression, max: Expression) = this(min, max, UnresolvedSeed) + + final override lazy val deterministic: Boolean = false + override val nodePatterns: Seq[TreePattern] = + Seq(RUNTIME_REPLACEABLE, EXPRESSION_WITH_RANDOM_SEED) + + override val dataType: DataType = { + val first = min.dataType + val second = max.dataType + (min.dataType, max.dataType) match { + case _ if !seedExpression.resolved || seedExpression.dataType == NullType => + NullType + case (_, NullType) | (NullType, _) => NullType + case (_, LongType) | (LongType, _) + if Seq(first, second).forall(integer) => LongType + case (_, IntegerType) | (IntegerType, _) + if Seq(first, second).forall(integer) => IntegerType + case (_, ShortType) | (ShortType, _) + if Seq(first, second).forall(integer) => ShortType + case (_, DoubleType) | (DoubleType, _) => DoubleType + case (_, FloatType) | (FloatType, _) => FloatType + case _ => + throw SparkException.internalError( + s"Unexpected argument data types: ${min.dataType}, ${max.dataType}") + } + } + + private def integer(t: DataType): Boolean = t match { + case _: ShortType | _: IntegerType | _: LongType => true + case _ => false + } + + override def checkInputDataTypes(): TypeCheckResult = { + var result: TypeCheckResult = TypeCheckResult.TypeCheckSuccess + def requiredType = "integer or floating-point" + Seq((min, "min", 0), + (max, "max", 1), + (seedExpression, "seed", 2)).foreach { + case (expr: Expression, name: String, index: Int) => + if (result == TypeCheckResult.TypeCheckSuccess) { + if (!expr.foldable) { + result = DataTypeMismatch( + errorSubClass = "NON_FOLDABLE_INPUT", + messageParameters = Map( + "inputName" -> name, + "inputType" -> requiredType, + "inputExpr" -> toSQLExpr(expr))) + } else expr.dataType match { + case _: ShortType | _: IntegerType | _: LongType | _: FloatType | _: DoubleType | + _: NullType => + case _ => + result = DataTypeMismatch( + errorSubClass = "UNEXPECTED_INPUT_TYPE", + messageParameters = Map( + "paramIndex" -> ordinalNumber(index), + "requiredType" -> requiredType, + "inputSql" -> toSQLExpr(expr), + "inputType" -> toSQLType(expr.dataType))) + } + } + } + result + } + + override def first: Expression = min + override def second: Expression = max + override def third: Expression = seedExpression + + override def withNewSeed(newSeed: Long): Expression = + Uniform(min, max, Literal(newSeed, LongType)) + + override def withNewChildrenInternal( + newFirst: Expression, newSecond: Expression, newThird: Expression): Expression = + Uniform(newFirst, newSecond, newThird) + + override def replacement: Expression = { + if (Seq(min, max, seedExpression).exists(_.dataType == NullType)) { + Literal(null) + } else { + def cast(e: Expression, to: DataType): Expression = if (e.dataType == to) e else Cast(e, to) + cast(Add( + cast(min, DoubleType), + Multiply( + Subtract( + cast(max, DoubleType), + cast(min, DoubleType)), + Rand(seed))), + dataType) + } + } +} + +@ExpressionDescription( + usage = """ + _FUNC_(length[, seed]) - Returns a string of the specified length whose characters are chosen + uniformly at random from the following pool of characters: 0-9, a-z, A-Z. The random seed is + optional. The string length must be a constant two-byte or four-byte integer (SMALLINT or INT, + respectively). + """, + examples = + """ + Examples: + > SELECT _FUNC_(3, 0) AS result; + ceV + """, + since = "4.0.0", + group = "string_funcs") +case class RandStr(length: Expression, override val seedExpression: Expression) + extends ExpressionWithRandomSeed with BinaryLike[Expression] with Nondeterministic { + def this(length: Expression) = this(length, UnresolvedSeed) + + override def nullable: Boolean = false + override def dataType: DataType = StringType + override def stateful: Boolean = true + override def left: Expression = length + override def right: Expression = seedExpression + + /** + * Record ID within each partition. By being transient, the Random Number Generator is + * reset every time we serialize and deserialize and initialize it. + */ + @transient protected var rng: XORShiftRandom = _ + + @transient protected lazy val seed: Long = seedExpression match { + case e if e.dataType == IntegerType => e.eval().asInstanceOf[Int] + case e if e.dataType == LongType => e.eval().asInstanceOf[Long] + } + override protected def initializeInternal(partitionIndex: Int): Unit = { + rng = new XORShiftRandom(seed + partitionIndex) + } + + override def withNewSeed(newSeed: Long): Expression = RandStr(length, Literal(newSeed, LongType)) + override def withNewChildrenInternal(newFirst: Expression, newSecond: Expression): Expression = + RandStr(newFirst, newSecond) + + override def checkInputDataTypes(): TypeCheckResult = { + var result: TypeCheckResult = TypeCheckResult.TypeCheckSuccess + def requiredType = "INT or SMALLINT" + Seq((length, "length", 0), + (seedExpression, "seedExpression", 1)).foreach { + case (expr: Expression, name: String, index: Int) => + if (result == TypeCheckResult.TypeCheckSuccess) { + if (!expr.foldable) { + result = DataTypeMismatch( + errorSubClass = "NON_FOLDABLE_INPUT", + messageParameters = Map( + "inputName" -> name, + "inputType" -> requiredType, + "inputExpr" -> toSQLExpr(expr))) + } else expr.dataType match { + case _: ShortType | _: IntegerType => + case _: LongType if index == 1 => + case _ => + result = DataTypeMismatch( + errorSubClass = "UNEXPECTED_INPUT_TYPE", + messageParameters = Map( + "paramIndex" -> ordinalNumber(index), + "requiredType" -> requiredType, + "inputSql" -> toSQLExpr(expr), + "inputType" -> toSQLType(expr.dataType))) + } + } + } + result + } + + override def evalInternal(input: InternalRow): Any = { + val numChars = length.eval(input).asInstanceOf[Number].intValue() + val bytes = new Array[Byte](numChars) + (0 until numChars).foreach { i => + // We generate a random number between 0 and 61, inclusive. Between the 62 different choices + // we choose 0-9, a-z, or A-Z, where each category comprises 10 choices, 26 choices, or 26 + // choices, respectively (10 + 26 + 26 = 62). + val num = (rng.nextInt() % 62).abs + num match { + case _ if num < 10 => + bytes.update(i, ('0' + num).toByte) + case _ if num < 36 => + bytes.update(i, ('a' + num - 10).toByte) + case _ => + bytes.update(i, ('A' + num - 36).toByte) + } + } + val result: UTF8String = UTF8String.fromBytes(bytes.toArray) + result + } + + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val className = classOf[XORShiftRandom].getName + val rngTerm = ctx.addMutableState(className, "rng") + ctx.addPartitionInitializationStatement( + s"$rngTerm = new $className(${seed}L + partitionIndex);") + val eval = length.genCode(ctx) + ev.copy(code = + code""" + |${eval.code} + |int length = (int)(${eval.value}); + |char[] chars = new char[length]; + |for (int i = 0; i < length; i++) { + | int v = Math.abs($rngTerm.nextInt() % 62); + | if (v < 10) { + | chars[i] = (char)('0' + v); + | } else if (v < 36) { + | chars[i] = (char)('a' + (v - 10)); + | } else { + | chars[i] = (char)('A' + (v - 36)); + | } + |} + |UTF8String ${ev.value} = UTF8String.fromString(new String(chars)); + |boolean ${ev.isNull} = false; + |""".stripMargin, + isNull = FalseLiteral) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RandomSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RandomSuite.scala index 2aa53f581555f..2d58d9d3136aa 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RandomSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RandomSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.types.{IntegerType, LongType} class RandomSuite extends SparkFunSuite with ExpressionEvalHelper { @@ -41,4 +42,27 @@ class RandomSuite extends SparkFunSuite with ExpressionEvalHelper { assert(Rand(Literal(1L), false).sql === "rand(1L)") assert(Randn(Literal(1L), false).sql === "randn(1L)") } + + test("SPARK-49505: Test the RANDSTR and UNIFORM SQL functions without codegen") { + // Note that we use a seed of zero in these tests to keep the results deterministic. + def testRandStr(first: Any, result: Any): Unit = { + checkEvaluationWithoutCodegen( + RandStr(Literal(first), Literal(0)), CatalystTypeConverters.convertToCatalyst(result)) + } + testRandStr(1, "c") + testRandStr(5, "ceV0P") + testRandStr(10, "ceV0PXaR2I") + testRandStr(10L, "ceV0PXaR2I") + + def testUniform(first: Any, second: Any, result: Any): Unit = { + checkEvaluationWithoutCodegen( + Uniform(Literal(first), Literal(second), Literal(0)).replacement, + CatalystTypeConverters.convertToCatalyst(result)) + } + testUniform(0, 1, 0) + testUniform(0, 10, 7) + testUniform(0L, 10L, 7L) + testUniform(10.0F, 20.0F, 17.604954F) + testUniform(10L, 20.0F, 17.604954F) + } } 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 f53b3874e6b8c..5ad1380e1fb82 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 @@ -265,6 +265,7 @@ | org.apache.spark.sql.catalyst.expressions.RaiseErrorExpressionBuilder | raise_error | SELECT raise_error('custom error message') | struct | | org.apache.spark.sql.catalyst.expressions.Rand | rand | SELECT rand() | struct | | org.apache.spark.sql.catalyst.expressions.Rand | random | SELECT random() | struct | +| org.apache.spark.sql.catalyst.expressions.RandStr | randstr | SELECT randstr(3, 0) AS result | struct | | org.apache.spark.sql.catalyst.expressions.Randn | randn | SELECT randn() | struct | | org.apache.spark.sql.catalyst.expressions.Rank | rank | SELECT a, b, rank(b) OVER (PARTITION BY a ORDER BY b) FROM VALUES ('A1', 2), ('A1', 1), ('A2', 3), ('A1', 1) tab(a, b) | struct | | org.apache.spark.sql.catalyst.expressions.RegExpCount | regexp_count | SELECT regexp_count('Steven Jones and Stephen Smith are the best players', 'Ste(v|ph)en') | struct | @@ -367,6 +368,7 @@ | 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.Uniform | uniform | SELECT uniform(10, 20, 0) > 0 AS result | 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 | diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/random.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/random.sql.out index 3cacbdc141053..133cd6a60a4fb 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/random.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/random.sql.out @@ -93,3 +93,404 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "fragment" : "rand('1')" } ] } + + +-- !query +SELECT uniform(0, 1, 0) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT uniform(0, 10, 0) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT uniform(0L, 10L, 0) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT uniform(0, 10L, 0) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT uniform(0, 10S, 0) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT uniform(10, 20, 0) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT uniform(10.0F, 20.0F, 0) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT uniform(10.0D, 20.0D, CAST(3 / 7 AS LONG)) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT uniform(10, 20.0F, 0) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT uniform(10, 20, 0) AS result FROM VALUES (0), (1), (2) tab(col) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT uniform(10, 20.0F) IS NOT NULL AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT uniform(NULL, 1, 0) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT uniform(0, NULL, 0) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT uniform(0, 1, NULL) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT uniform(10, 20, col) AS result FROM VALUES (0), (1), (2) tab(col) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"col\"", + "inputName" : "seed", + "inputType" : "integer or floating-point", + "sqlExpr" : "\"uniform(10, 20, col)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "uniform(10, 20, col)" + } ] +} + + +-- !query +SELECT uniform(col, 10, 0) AS result FROM VALUES (0), (1), (2) tab(col) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"col\"", + "inputName" : "min", + "inputType" : "integer or floating-point", + "sqlExpr" : "\"uniform(col, 10, 0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "uniform(col, 10, 0)" + } ] +} + + +-- !query +SELECT uniform(10) AS result +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "1", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3]", + "functionName" : "`uniform`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 18, + "fragment" : "uniform(10)" + } ] +} + + +-- !query +SELECT uniform(10, 20, 30, 40) AS result +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "4", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3]", + "functionName" : "`uniform`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "uniform(10, 20, 30, 40)" + } ] +} + + +-- !query +SELECT randstr(1, 0) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT randstr(5, 0) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT randstr(10, 0) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT randstr(10S, 0) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT randstr(10, 0) AS result FROM VALUES (0), (1), (2) tab(col) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT randstr(10) IS NOT NULL AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT randstr(10L, 0) AS result +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"10\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "first", + "requiredType" : "INT or SMALLINT", + "sqlExpr" : "\"randstr(10, 0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "randstr(10L, 0)" + } ] +} + + +-- !query +SELECT randstr(10.0F, 0) AS result +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"10.0\"", + "inputType" : "\"FLOAT\"", + "paramIndex" : "first", + "requiredType" : "INT or SMALLINT", + "sqlExpr" : "\"randstr(10.0, 0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "randstr(10.0F, 0)" + } ] +} + + +-- !query +SELECT randstr(10.0D, 0) AS result +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"10.0\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "first", + "requiredType" : "INT or SMALLINT", + "sqlExpr" : "\"randstr(10.0, 0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "randstr(10.0D, 0)" + } ] +} + + +-- !query +SELECT randstr(NULL, 0) AS result +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"NULL\"", + "inputType" : "\"VOID\"", + "paramIndex" : "first", + "requiredType" : "INT or SMALLINT", + "sqlExpr" : "\"randstr(NULL, 0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "randstr(NULL, 0)" + } ] +} + + +-- !query +SELECT randstr(0, NULL) AS result +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"NULL\"", + "inputType" : "\"VOID\"", + "paramIndex" : "second", + "requiredType" : "INT or SMALLINT", + "sqlExpr" : "\"randstr(0, NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "randstr(0, NULL)" + } ] +} + + +-- !query +SELECT randstr(col, 0) AS result FROM VALUES (0), (1), (2) tab(col) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"col\"", + "inputName" : "length", + "inputType" : "INT or SMALLINT", + "sqlExpr" : "\"randstr(col, 0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "randstr(col, 0)" + } ] +} + + +-- !query +SELECT randstr(10, col) AS result FROM VALUES (0), (1), (2) tab(col) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"col\"", + "inputName" : "seedExpression", + "inputType" : "INT or SMALLINT", + "sqlExpr" : "\"randstr(10, col)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "randstr(10, col)" + } ] +} + + +-- !query +SELECT randstr(10, 0, 1) AS result +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[1, 2]", + "functionName" : "`randstr`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "randstr(10, 0, 1)" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/inputs/random.sql b/sql/core/src/test/resources/sql-tests/inputs/random.sql index a1aae7b8759dc..a71b0293295fc 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/random.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/random.sql @@ -14,4 +14,44 @@ SELECT randn(NULL); SELECT randn(cast(NULL AS long)); -- randn unsupported data type -SELECT rand('1') +SELECT rand('1'); + +-- The uniform random number generation function supports generating random numbers within a +-- specified range. We use a seed of zero for these queries to keep tests deterministic. +SELECT uniform(0, 1, 0) AS result; +SELECT uniform(0, 10, 0) AS result; +SELECT uniform(0L, 10L, 0) AS result; +SELECT uniform(0, 10L, 0) AS result; +SELECT uniform(0, 10S, 0) AS result; +SELECT uniform(10, 20, 0) AS result; +SELECT uniform(10.0F, 20.0F, 0) AS result; +SELECT uniform(10.0D, 20.0D, CAST(3 / 7 AS LONG)) AS result; +SELECT uniform(10, 20.0F, 0) AS result; +SELECT uniform(10, 20, 0) AS result FROM VALUES (0), (1), (2) tab(col); +SELECT uniform(10, 20.0F) IS NOT NULL AS result; +-- Negative test cases for the uniform random number generator. +SELECT uniform(NULL, 1, 0) AS result; +SELECT uniform(0, NULL, 0) AS result; +SELECT uniform(0, 1, NULL) AS result; +SELECT uniform(10, 20, col) AS result FROM VALUES (0), (1), (2) tab(col); +SELECT uniform(col, 10, 0) AS result FROM VALUES (0), (1), (2) tab(col); +SELECT uniform(10) AS result; +SELECT uniform(10, 20, 30, 40) AS result; + +-- The randstr random string generation function supports generating random strings within a +-- specified length. We use a seed of zero for these queries to keep tests deterministic. +SELECT randstr(1, 0) AS result; +SELECT randstr(5, 0) AS result; +SELECT randstr(10, 0) AS result; +SELECT randstr(10S, 0) AS result; +SELECT randstr(10, 0) AS result FROM VALUES (0), (1), (2) tab(col); +SELECT randstr(10) IS NOT NULL AS result; +-- Negative test cases for the randstr random number generator. +SELECT randstr(10L, 0) AS result; +SELECT randstr(10.0F, 0) AS result; +SELECT randstr(10.0D, 0) AS result; +SELECT randstr(NULL, 0) AS result; +SELECT randstr(0, NULL) AS result; +SELECT randstr(col, 0) AS result FROM VALUES (0), (1), (2) tab(col); +SELECT randstr(10, col) AS result FROM VALUES (0), (1), (2) tab(col); +SELECT randstr(10, 0, 1) AS result; diff --git a/sql/core/src/test/resources/sql-tests/results/random.sql.out b/sql/core/src/test/resources/sql-tests/results/random.sql.out index 16984de3ff257..0b4e5e078ee15 100644 --- a/sql/core/src/test/resources/sql-tests/results/random.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/random.sql.out @@ -113,3 +113,472 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "fragment" : "rand('1')" } ] } + + +-- !query +SELECT uniform(0, 1, 0) AS result +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT uniform(0, 10, 0) AS result +-- !query schema +struct +-- !query output +7 + + +-- !query +SELECT uniform(0L, 10L, 0) AS result +-- !query schema +struct +-- !query output +7 + + +-- !query +SELECT uniform(0, 10L, 0) AS result +-- !query schema +struct +-- !query output +7 + + +-- !query +SELECT uniform(0, 10S, 0) AS result +-- !query schema +struct +-- !query output +7 + + +-- !query +SELECT uniform(10, 20, 0) AS result +-- !query schema +struct +-- !query output +17 + + +-- !query +SELECT uniform(10.0F, 20.0F, 0) AS result +-- !query schema +struct +-- !query output +17.604954 + + +-- !query +SELECT uniform(10.0D, 20.0D, CAST(3 / 7 AS LONG)) AS result +-- !query schema +struct +-- !query output +17.604953758285916 + + +-- !query +SELECT uniform(10, 20.0F, 0) AS result +-- !query schema +struct +-- !query output +17.604954 + + +-- !query +SELECT uniform(10, 20, 0) AS result FROM VALUES (0), (1), (2) tab(col) +-- !query schema +struct +-- !query output +15 +16 +17 + + +-- !query +SELECT uniform(10, 20.0F) IS NOT NULL AS result +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT uniform(NULL, 1, 0) AS result +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT uniform(0, NULL, 0) AS result +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT uniform(0, 1, NULL) AS result +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT uniform(10, 20, col) AS result FROM VALUES (0), (1), (2) tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"col\"", + "inputName" : "seed", + "inputType" : "integer or floating-point", + "sqlExpr" : "\"uniform(10, 20, col)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "uniform(10, 20, col)" + } ] +} + + +-- !query +SELECT uniform(col, 10, 0) AS result FROM VALUES (0), (1), (2) tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"col\"", + "inputName" : "min", + "inputType" : "integer or floating-point", + "sqlExpr" : "\"uniform(col, 10, 0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "uniform(col, 10, 0)" + } ] +} + + +-- !query +SELECT uniform(10) AS result +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "1", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3]", + "functionName" : "`uniform`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 18, + "fragment" : "uniform(10)" + } ] +} + + +-- !query +SELECT uniform(10, 20, 30, 40) AS result +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "4", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3]", + "functionName" : "`uniform`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "uniform(10, 20, 30, 40)" + } ] +} + + +-- !query +SELECT randstr(1, 0) AS result +-- !query schema +struct +-- !query output +c + + +-- !query +SELECT randstr(5, 0) AS result +-- !query schema +struct +-- !query output +ceV0P + + +-- !query +SELECT randstr(10, 0) AS result +-- !query schema +struct +-- !query output +ceV0PXaR2I + + +-- !query +SELECT randstr(10S, 0) AS result +-- !query schema +struct +-- !query output +ceV0PXaR2I + + +-- !query +SELECT randstr(10, 0) AS result FROM VALUES (0), (1), (2) tab(col) +-- !query schema +struct +-- !query output +ceV0PXaR2I +fYxVfArnv7 +iSIv0VT2XL + + +-- !query +SELECT randstr(10) IS NOT NULL AS result +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT randstr(10L, 0) AS result +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"10\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "first", + "requiredType" : "INT or SMALLINT", + "sqlExpr" : "\"randstr(10, 0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "randstr(10L, 0)" + } ] +} + + +-- !query +SELECT randstr(10.0F, 0) AS result +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"10.0\"", + "inputType" : "\"FLOAT\"", + "paramIndex" : "first", + "requiredType" : "INT or SMALLINT", + "sqlExpr" : "\"randstr(10.0, 0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "randstr(10.0F, 0)" + } ] +} + + +-- !query +SELECT randstr(10.0D, 0) AS result +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"10.0\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "first", + "requiredType" : "INT or SMALLINT", + "sqlExpr" : "\"randstr(10.0, 0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "randstr(10.0D, 0)" + } ] +} + + +-- !query +SELECT randstr(NULL, 0) AS result +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"NULL\"", + "inputType" : "\"VOID\"", + "paramIndex" : "first", + "requiredType" : "INT or SMALLINT", + "sqlExpr" : "\"randstr(NULL, 0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "randstr(NULL, 0)" + } ] +} + + +-- !query +SELECT randstr(0, NULL) AS result +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"NULL\"", + "inputType" : "\"VOID\"", + "paramIndex" : "second", + "requiredType" : "INT or SMALLINT", + "sqlExpr" : "\"randstr(0, NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "randstr(0, NULL)" + } ] +} + + +-- !query +SELECT randstr(col, 0) AS result FROM VALUES (0), (1), (2) tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"col\"", + "inputName" : "length", + "inputType" : "INT or SMALLINT", + "sqlExpr" : "\"randstr(col, 0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "randstr(col, 0)" + } ] +} + + +-- !query +SELECT randstr(10, col) AS result FROM VALUES (0), (1), (2) tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"col\"", + "inputName" : "seedExpression", + "inputType" : "INT or SMALLINT", + "sqlExpr" : "\"randstr(10, col)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "randstr(10, col)" + } ] +} + + +-- !query +SELECT randstr(10, 0, 1) AS result +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[1, 2]", + "functionName" : "`randstr`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "randstr(10, 0, 1)" + } ] +}