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

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -19,15 +19,15 @@ package org.apache.spark.sql.catalyst.analysis

import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.expressions.Cast
import org.apache.spark.sql.catalyst.expressions.postgreSQL.PostgreCastToBoolean
import org.apache.spark.sql.catalyst.expressions.postgreSQL.{PostgreCastToBoolean, PostgreCastToLong}
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.{BooleanType, StringType}
import org.apache.spark.sql.types.{BooleanType, LongType, StringType}

object PostgreSQLDialect {
val postgreSQLDialectRules: List[Rule[LogicalPlan]] =
CastToBoolean ::
CastToBoolean :: CastToLong ::
Nil

object CastToBoolean extends Rule[LogicalPlan] with Logging {
Expand All @@ -46,4 +46,21 @@ object PostgreSQLDialect {
}
}
}

object CastToLong extends Rule[LogicalPlan] with Logging {
override def apply(plan: LogicalPlan): LogicalPlan = {
// The SQL configuration `spark.sql.dialect` can be changed in runtime.
// To make sure the configuration is effective, we have to check it during rule execution.
val conf = SQLConf.get
if (conf.usePostgreSQLDialect) {
plan.transformExpressions {
case Cast(child, dataType, timeZoneId)
if child.dataType != LongType && dataType == LongType =>
PostgreCastToLong(child, timeZoneId)
}
} else {
plan
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -471,7 +471,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit
}

// LongConverter
private[this] def castToLong(from: DataType): Any => Any = from match {
protected[this] def castToLong(from: DataType): Any => Any = from match {
case StringType =>
val result = new LongWrapper()
buildCast[UTF8String](_, s => if (s.toLong(result)) result.value else null)
Expand Down Expand Up @@ -1422,7 +1422,9 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit
(c, evPrim, evNull) => code"$evPrim = (int) $c;"
}

private[this] def castToLongCode(from: DataType, ctx: CodegenContext): CastFunction = from match {
protected[this] def castToLongCode(
from: DataType,
ctx: CodegenContext): CastFunction = from match {
case StringType =>
val wrapper = ctx.freshVariable("longWrapper", classOf[UTF8String.LongWrapper])

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,76 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.sql.catalyst.expressions.postgreSQL

import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
import org.apache.spark.sql.catalyst.expressions.{CastBase, Expression, TimeZoneAwareExpression}
import org.apache.spark.sql.catalyst.expressions.codegen.Block._
import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String
import org.apache.spark.unsafe.types.UTF8String.LongWrapper

case class PostgreCastToLong(child: Expression, timeZoneId: Option[String])
extends CastBase {
override def dataType: DataType = LongType

override protected def ansiEnabled: Boolean =
throw new AnalysisException("")

override def checkInputDataTypes(): TypeCheckResult = child.dataType match {
case DateType | TimestampType | NullType =>
TypeCheckResult.TypeCheckFailure(s"cannot cast type ${child.dataType} to long")
case _ =>
TypeCheckResult.TypeCheckSuccess
}
/** Returns a copy of this expression with the specified timeZoneId. */
override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression =
copy(timeZoneId = Option(timeZoneId))

override def castToLong(from: DataType): Any => Any = from match {
case StringType =>
val result = new LongWrapper()
buildCast[UTF8String](_, s => if (s.toLong(result)) result.value
else throw new AnalysisException(s"invalid input syntax for type long: $s"))
case BooleanType =>
buildCast[Boolean](_, b => if (b) 1L else 0L)
case x: NumericType =>
b => x.numeric.asInstanceOf[Numeric[Any]].toLong(b)
}

override def castToLongCode(from: DataType, ctx: CodegenContext): CastFunction = from match {
case StringType =>
val wrapper = ctx.freshVariable("longWrapper", classOf[UTF8String.LongWrapper])
(c, evPrim, _) =>
code"""
UTF8String.LongWrapper $wrapper = new UTF8String.LongWrapper();
if ($c.toLong($wrapper)) {
$evPrim = $wrapper.value;
} else {
throw new AnalysisException(s"invalid input syntax for type long: $c");
}
$wrapper = null;
"""
case BooleanType =>
(c, evPrim, _) => code"$evPrim = $c ? 1L : 0L;"
case DecimalType() =>
(c, evPrim, _) => code"$evPrim = $c.to${"long".capitalize}();"
case NumericType() =>
(c, evPrim, _) => code"$evPrim = (long) $c;"
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -70,4 +70,8 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper {
assert(PostgreCastToBoolean(Literal(1.toDouble), None).checkInputDataTypes().isFailure)
assert(PostgreCastToBoolean(Literal(1.toFloat), None).checkInputDataTypes().isFailure)
}

test("unsupported data types to cast to long") {
// TODO: Test cases to be added
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ package org.apache.spark.sql
import org.apache.spark.SparkConf
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.DateType

class PostgreSQLDialectQuerySuite extends QueryTest with SharedSparkSession {

Expand All @@ -39,4 +40,8 @@ class PostgreSQLDialectQuerySuite extends QueryTest with SharedSparkSession {
intercept[IllegalArgumentException](sql(s"select cast('$input' as boolean)").collect())
}
}

test("cast to long") {
// TODO: Add test cases
}
}