Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
32 commits
Select commit Hold shift + click to select a range
0b293db
[SPARK-29774][SQL] Date and Timestamp type +/- null should be null as…
yaooqinn Nov 6, 2019
f726297
Merge branch 'master' into SPARK-29774
yaooqinn Nov 27, 2019
e7225a3
regen golden file
yaooqinn Nov 27, 2019
b925517
null - dates
yaooqinn Nov 27, 2019
cd49411
Merge branch 'master' into SPARK-29774
yaooqinn Dec 2, 2019
57b13e9
support +/-
yaooqinn Dec 2, 2019
eab6a83
support ×/÷
yaooqinn Dec 2, 2019
e8b75ba
import
yaooqinn Dec 2, 2019
02b3738
childResolved required
yaooqinn Dec 2, 2019
e89d806
regen golden file
yaooqinn Dec 2, 2019
0694e07
update comments
yaooqinn Dec 2, 2019
0f5618b
fix tests
yaooqinn Dec 3, 2019
efab3ec
fix tests
yaooqinn Dec 3, 2019
1c27be1
refine case match pattern
yaooqinn Dec 3, 2019
5df6980
fix ut
yaooqinn Dec 3, 2019
9817d2d
hack assert Equal
yaooqinn Dec 3, 2019
9808b9c
regen g f
yaooqinn Dec 3, 2019
b190612
AnalysisTest
yaooqinn Dec 3, 2019
e544137
regen g f
yaooqinn Dec 3, 2019
83705fd
fix test
yaooqinn Dec 4, 2019
846802d
date add/sub only work for int/smallint/tinyint
yaooqinn Dec 4, 2019
4af7edb
regen g f
yaooqinn Dec 4, 2019
a67be30
refine
yaooqinn Dec 4, 2019
9a1affd
type coercion for subtract timestamp
yaooqinn Dec 4, 2019
ae70022
add and reorgnize tests in datetime.sql
yaooqinn Dec 4, 2019
571225b
DateExpressionsSuite
yaooqinn Dec 4, 2019
6052e5a
fix py
yaooqinn Dec 4, 2019
928fd86
fix py
yaooqinn Dec 4, 2019
254d2d2
Revert "fix py"
yaooqinn Dec 5, 2019
5dd632c
fix py
yaooqinn Dec 5, 2019
c84d46e
rm unresolved binary arithmetic
yaooqinn Dec 5, 2019
a44948e
import
yaooqinn Dec 5, 2019
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
rm unresolved binary arithmetic
  • Loading branch information
yaooqinn committed Dec 5, 2019
commit c84d46ea6d384dcb1f442ca54abad48e59c92bb3
Original file line number Diff line number Diff line change
Expand Up @@ -248,41 +248,41 @@ class Analyzer(
)

/**
* For [[UnresolvedAdd]]:
* 1. if both side are interval, turns it to [[Add]];
* For [[Add]]:
* 1. if both side are interval, stays the same;
* 2. else if one side is interval, turns it to [[TimeAdd]];
* 3. else if one side is date, turns it to [[DateAdd]] ;
* 4. else turns it to [[Add]].
* 4. else stays the same.
*
* For [[UnresolvedSubtract]]:
* 1. if both side are interval, turns it to [[Subtract]];
* For [[Subtract]]:
* 1. if both side are interval, stays the same;
* 2. else if the right side is an interval, turns it to [[TimeSub]];
* 3. else if one side is timestamp, turns it to [[SubtractTimestamps]];
* 4. else if the right side is date, turns it to [[DateDiff]]/[[SubtractDates]];
* 5. else if the left side is date, turns it to [[DateSub]];
* 6. else turns it to [[Subtract]].
* 6. else turns it to stays the same.
*
* For [[UnresolvedMultiply]]:
* For [[Multiply]]:
* 1. If one side is interval, turns it to [[MultiplyInterval]];
* 2. otherwise, turns it to [[Multiply]].
* 2. otherwise, stays the same.
*
* For [[UnresolvedDivide]]:
* For [[Divide]]:
* 1. If the left side is interval, turns it to [[DivideInterval]];
* 2. otherwise, turns it to [[Divide]].
* 2. otherwise, stays the same.
*/
case class ResolveBinaryArithmetic(conf: SQLConf) extends Rule[LogicalPlan] {
override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
case p: LogicalPlan => p.transformExpressionsUp {
case u @ UnresolvedAdd(l, r) if u.childrenResolved => (l.dataType, r.dataType) match {
case (CalendarIntervalType, CalendarIntervalType) => Add(l, r)
case a @ Add(l, r) if a.childrenResolved => (l.dataType, r.dataType) match {
case (CalendarIntervalType, CalendarIntervalType) => a
case (_, CalendarIntervalType) => Cast(TimeAdd(l, r), l.dataType)
case (CalendarIntervalType, _) => Cast(TimeAdd(r, l), r.dataType)
case (DateType, _) => DateAdd(l, r)
case (_, DateType) => DateAdd(r, l)
case _ => Add(l, r)
case _ => a
}
case u @ UnresolvedSubtract(l, r) if u.childrenResolved => (l.dataType, r.dataType) match {
case (CalendarIntervalType, CalendarIntervalType) => Subtract(l, r)
case s @ Subtract(l, r) if s.childrenResolved => (l.dataType, r.dataType) match {
case (CalendarIntervalType, CalendarIntervalType) => s
case (_, CalendarIntervalType) => Cast(TimeSub(l, r), l.dataType)
case (TimestampType, _) => SubtractTimestamps(l, r)
case (_, TimestampType) => SubtractTimestamps(l, r)
Expand All @@ -292,16 +292,16 @@ class Analyzer(
SubtractDates(l, r)
}
case (DateType, _) => DateSub(l, r)
case _ => Subtract(l, r)
case _ => s
}
case u @ UnresolvedMultiply(l, r) if u.childrenResolved => (l.dataType, r.dataType) match {
case m @ Multiply(l, r) if m.childrenResolved => (l.dataType, r.dataType) match {
case (CalendarIntervalType, _) => MultiplyInterval(l, r)
case (_, CalendarIntervalType) => MultiplyInterval(r, l)
case _ => Multiply(l, r)
case _ => m
}
case u @ UnresolvedDivide(l, r) if u.childrenResolved => (l.dataType, r.dataType) match {
case d @ Divide(l, r) if d.childrenResolved => (l.dataType, r.dataType) match {
case (CalendarIntervalType, _) => DivideInterval(l, r)
case _ => Divide(l, r)
case _ => d
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -544,23 +544,3 @@ case class UnresolvedOrdinal(ordinal: Int)
override def nullable: Boolean = throw new UnresolvedException(this, "nullable")
override lazy val resolved = false
}

abstract class UnresolvedBinaryExpression(operator: String)
extends BinaryExpression with Unevaluable {
override lazy val resolved: Boolean = false
override def dataType: DataType = throw new UnresolvedException(this, "dataType")
override def sql: String = s"${left.sql} $operator ${right.sql}"
}

case class UnresolvedAdd(left: Expression, right: Expression)
extends UnresolvedBinaryExpression("+") {
}

case class UnresolvedSubtract(left: Expression, right: Expression)
extends UnresolvedBinaryExpression("-")

case class UnresolvedMultiply(left: Expression, right: Expression)
extends UnresolvedBinaryExpression("*")

case class UnresolvedDivide(left: Expression, right: Expression)
extends UnresolvedBinaryExpression("/")
Original file line number Diff line number Diff line change
Expand Up @@ -1429,17 +1429,17 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
val right = expression(ctx.right)
ctx.operator.getType match {
case SqlBaseParser.ASTERISK =>
UnresolvedMultiply(left, right)
Multiply(left, right)
case SqlBaseParser.SLASH =>
UnresolvedDivide(left, right)
Divide(left, right)
case SqlBaseParser.PERCENT =>
Remainder(left, right)
case SqlBaseParser.DIV =>
IntegralDivide(left, right)
case SqlBaseParser.PLUS =>
UnresolvedAdd(left, right)
Add(left, right)
case SqlBaseParser.MINUS =>
UnresolvedSubtract(left, right)
Subtract(left, right)
case SqlBaseParser.CONCAT_PIPE =>
Concat(left :: right :: Nil)
case SqlBaseParser.AMPERSAND =>
Expand Down Expand Up @@ -1696,12 +1696,9 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
*/
override def visitFrameBound(ctx: FrameBoundContext): Expression = withOrigin(ctx) {
def value: Expression = {
expression(ctx.expression) match {
case u: UnresolvedBinaryExpression if u.childrenResolved && u.foldable => u
case e =>
validate(e.resolved && e.foldable, "Frame bound value must be a literal.", ctx)
e
}
val e = expression(ctx.expression)
validate(e.resolved && e.foldable, "Frame bound value must be a literal.", ctx)
e
}

ctx.boundType.getType match {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@ import java.nio.charset.StandardCharsets.UTF_8
import java.util.concurrent.atomic.AtomicBoolean

import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.analysis.UnresolvedBinaryExpression
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.{NumericType, StringType}
Expand Down Expand Up @@ -143,10 +142,7 @@ package object util extends Logging {
"`" + name.replace("`", "``") + "`"
}

def toPrettySQL(e: Expression): String = e match {
case u: UnresolvedBinaryExpression => u.sql
case _ => usePrettyExpression(e).sql
}
def toPrettySQL(e: Expression): String = usePrettyExpression(e).sql

def escapeSingleQuotedString(str: String): String = {
val builder = StringBuilder.newBuilder
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,29 +16,19 @@
*/
package org.apache.spark.sql.catalyst.parser

import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedAdd, UnresolvedDivide, UnresolvedMultiply, UnresolvedSubtract}
import org.apache.spark.sql.catalyst.expressions.{Add, Divide, Multiply, Subtract}
import org.apache.spark.sql.catalyst.analysis.AnalysisTest
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan

/**
* Test various parser errors.
*/
class ErrorParserSuite extends AnalysisTest {
import CatalystSqlParser._
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.dsl.plans._
import org.apache.spark.sql.catalyst.parser.CatalystSqlParser._
Copy link
Contributor

Choose a reason for hiding this comment

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

unnecessary change


private def assertEqual(sqlCommand: String, plan: LogicalPlan): Unit = {
val resolvedPlan = parsePlan(sqlCommand) resolveOperatorsUp {
case p: LogicalPlan => p transformAllExpressions {
case UnresolvedAdd(l, r) => Add(l, r)
case UnresolvedSubtract(l, r) => Subtract(l, r)
case UnresolvedMultiply(l, r) => Multiply(l, r)
case UnresolvedDivide(l, r) => Divide(l, r)
case other => other
}
}
assert(resolvedPlan == plan)
assert(parsePlan(sqlCommand) == plan)
}

def intercept(sqlCommand: String, messages: String*): Unit =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,15 +51,7 @@ class ExpressionParserSuite extends AnalysisTest {
sqlCommand: String,
e: Expression,
parser: ParserInterface = defaultParser): Unit = {
// usage for UnresolvedAdd etc here is just for tests
val expression = parser.parseExpression(sqlCommand) transform {
case UnresolvedAdd(l, r) => Add(l, r)
case UnresolvedSubtract(l, r) => Subtract(l, r)
case UnresolvedMultiply(l, r) => Multiply(l, r)
case UnresolvedDivide(l, r) => Divide(l, r)
case other => other
}
compareExpressions(expression, e)
compareExpressions(parser.parseExpression(sqlCommand), e)
}

private def intercept(sqlCommand: String, messages: String*): Unit =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
package org.apache.spark.sql.catalyst.parser

import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier}
import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedAlias, UnresolvedAttribute, UnresolvedDivide, UnresolvedFunction, UnresolvedGenerator, UnresolvedInlineTable, UnresolvedRelation, UnresolvedSubqueryColumnAliases, UnresolvedTableValuedFunction}
import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedAlias, UnresolvedAttribute, UnresolvedFunction, UnresolvedGenerator, UnresolvedInlineTable, UnresolvedRelation, UnresolvedSubqueryColumnAliases, UnresolvedTableValuedFunction}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical._
Expand All @@ -31,9 +31,9 @@ import org.apache.spark.sql.types.IntegerType
* There is also SparkSqlParserSuite in sql/core module for parser rules defined in sql/core module.
*/
class PlanParserSuite extends AnalysisTest {
import CatalystSqlParser._
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.dsl.plans._
import org.apache.spark.sql.catalyst.parser.CatalystSqlParser._
Copy link
Contributor

Choose a reason for hiding this comment

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

ditto


private def assertEqual(sqlCommand: String, plan: LogicalPlan): Unit = {
comparePlans(parsePlan(sqlCommand), plan, checkAnalysis = false)
Expand Down Expand Up @@ -240,8 +240,7 @@ class PlanParserSuite extends AnalysisTest {
val sql = "select * from t"
val plan = table("t").select(star())
assertEqual(s"$sql limit 10", plan.limit(10))
assertEqual(s"$sql limit cast(9 / 4 as int)",
plan.limit(Cast(UnresolvedDivide(9, 4), IntegerType)))
assertEqual(s"$sql limit cast(9 / 4 as int)", plan.limit(Cast(Literal(9) / 4, IntegerType)))
}

test("window spec") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,7 @@ struct<>
-- !query 6
select id, a+b, a-b, a*b, a/b from decimals_test order by id
-- !query 6 schema
struct<id:int,(CAST(a AS DECIMAL(38,17)) + CAST(b AS DECIMAL(38,17))):decimal(38,17),(CAST(a AS DECIMAL(38,17)) - CAST(b AS DECIMAL(38,17))):decimal(38,17),(CAST(a AS DECIMAL(38,18)) * CAST(b AS DECIMAL(38,18))):decimal(38,6),(CAST(a AS DECIMAL(38,18)) / CAST(b AS DECIMAL(38,18))):decimal(38,6)>
struct<id:int,(a + b):decimal(38,17),(a - b):decimal(38,17),(a * b):decimal(38,6),(a / b):decimal(38,6)>
-- !query 6 output
1 1099.00000000000000000 -899.00000000000000000 99900.000000 0.100100
2 24690.24600000000000000 0.00000000000000000 152402061.885129 1.000000
Expand Down Expand Up @@ -140,7 +140,7 @@ NULL
-- !query 16
select 12345678901234567890.0 * 12345678901234567890.0
-- !query 16 schema
struct<(CAST(12345678901234567890.0 AS DECIMAL(21,1)) * CAST(12345678901234567890.0 AS DECIMAL(21,1))):decimal(38,2)>
struct<(12345678901234567890.0 * 12345678901234567890.0):decimal(38,2)>
-- !query 16 output
NULL

Expand Down Expand Up @@ -196,7 +196,7 @@ spark.sql.decimalOperations.allowPrecisionLoss false
-- !query 23
select id, a+b, a-b, a*b, a/b from decimals_test order by id
-- !query 23 schema
struct<id:int,(CAST(a AS DECIMAL(38,18)) + CAST(b AS DECIMAL(38,18))):decimal(38,18),(CAST(a AS DECIMAL(38,18)) - CAST(b AS DECIMAL(38,18))):decimal(38,18),(CAST(a AS DECIMAL(38,18)) * CAST(b AS DECIMAL(38,18))):decimal(38,36),(CAST(a AS DECIMAL(38,18)) / CAST(b AS DECIMAL(38,18))):decimal(38,18)>
struct<id:int,(a + b):decimal(38,18),(a - b):decimal(38,18),(a * b):decimal(38,36),(a / b):decimal(38,18)>
-- !query 23 output
1 1099.000000000000000000 -899.000000000000000000 NULL 0.100100100100100100
2 24690.246000000000000000 0.000000000000000000 NULL 1.000000000000000000
Expand Down Expand Up @@ -282,7 +282,7 @@ NULL
-- !query 33
select 12345678901234567890.0 * 12345678901234567890.0
-- !query 33 schema
struct<(CAST(12345678901234567890.0 AS DECIMAL(21,1)) * CAST(12345678901234567890.0 AS DECIMAL(21,1))):decimal(38,2)>
struct<(12345678901234567890.0 * 12345678901234567890.0):decimal(38,2)>
-- !query 33 output
NULL

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -221,7 +221,7 @@ struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0)) + CAST(CAST(1 AS DECIMAL
-- !query 27
SELECT cast(1 as decimal(10, 0)) + cast(1 as decimal(10, 0)) FROM t
-- !query 27 schema
struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0)) + CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0))):decimal(11,0)>
struct<(CAST(1 AS DECIMAL(10,0)) + CAST(1 AS DECIMAL(10,0))):decimal(11,0)>
-- !query 27 output
2

Expand Down Expand Up @@ -553,7 +553,7 @@ struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(11,0)) + CAST(CAST(1 AS DECIMAL(
-- !query 67
SELECT cast(1 as decimal(10, 0)) + cast(1 as decimal(10, 0)) FROM t
-- !query 67 schema
struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0)) + CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0))):decimal(11,0)>
struct<(CAST(1 AS DECIMAL(10,0)) + CAST(1 AS DECIMAL(10,0))):decimal(11,0)>
-- !query 67 output
2

Expand Down Expand Up @@ -953,7 +953,7 @@ struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0)) - CAST(CAST(1 AS DECIMAL
-- !query 115
SELECT cast(1 as decimal(10, 0)) - cast(1 as decimal(10, 0)) FROM t
-- !query 115 schema
struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0)) - CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0))):decimal(11,0)>
struct<(CAST(1 AS DECIMAL(10,0)) - CAST(1 AS DECIMAL(10,0))):decimal(11,0)>
-- !query 115 output
0

Expand Down Expand Up @@ -1285,7 +1285,7 @@ struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(11,0)) - CAST(CAST(1 AS DECIMAL(
-- !query 155
SELECT cast(1 as decimal(10, 0)) - cast(1 as decimal(10, 0)) FROM t
-- !query 155 schema
struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0)) - CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0))):decimal(11,0)>
struct<(CAST(1 AS DECIMAL(10,0)) - CAST(1 AS DECIMAL(10,0))):decimal(11,0)>
-- !query 155 output
0

Expand Down Expand Up @@ -1685,7 +1685,7 @@ struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) * CAST(CAST(1 AS DECIMAL
-- !query 203
SELECT cast(1 as decimal(10, 0)) * cast(1 as decimal(10, 0)) FROM t
-- !query 203 schema
struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) * CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(21,0)>
struct<(CAST(1 AS DECIMAL(10,0)) * CAST(1 AS DECIMAL(10,0))):decimal(21,0)>
-- !query 203 output
1

Expand Down Expand Up @@ -2017,7 +2017,7 @@ struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) * CAST(CAST(1 AS DECIMAL(
-- !query 243
SELECT cast(1 as decimal(10, 0)) * cast(1 as decimal(10, 0)) FROM t
-- !query 243 schema
struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) * CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(21,0)>
struct<(CAST(1 AS DECIMAL(10,0)) * CAST(1 AS DECIMAL(10,0))):decimal(21,0)>
-- !query 243 output
1

Expand Down Expand Up @@ -2417,7 +2417,7 @@ struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL
-- !query 291
SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(10, 0)) FROM t
-- !query 291 schema
struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(21,11)>
struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(10,0))):decimal(21,11)>
-- !query 291 output
1.00000000000

Expand Down Expand Up @@ -2749,7 +2749,7 @@ struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(
-- !query 331
SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(10, 0)) FROM t
-- !query 331 schema
struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(21,11)>
struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(10,0))):decimal(21,11)>
-- !query 331 output
1.00000000000

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -661,7 +661,7 @@ struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) / CAST(1 AS DOUBLE)):double>
-- !query 79
SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(10, 0)) FROM t
-- !query 79 schema
struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(21,11)>
struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(10,0))):decimal(21,11)>
-- !query 79 output
1.00000000000

Expand Down