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 @@ -164,6 +164,9 @@ object Cast extends QueryErrorsBase {

case (udt1: UserDefinedType[_], udt2: UserDefinedType[_]) if udt2.acceptsType(udt1) => true

// Casts from concrete GEOGRAPHY(srid) to mixed GEOGRAPHY(ANY) is allowed.
case (gt1: GeographyType, gt2: GeographyType) if !gt1.isMixedSrid && gt2.isMixedSrid =>
true
// Casting from GEOGRAPHY to GEOMETRY with the same SRID is allowed.
case (geog: GeographyType, geom: GeometryType) if geog.srid == geom.srid =>
true
Expand Down Expand Up @@ -294,6 +297,9 @@ object Cast extends QueryErrorsBase {

case (udt1: UserDefinedType[_], udt2: UserDefinedType[_]) if udt2.acceptsType(udt1) => true

// Casts from concrete GEOGRAPHY(srid) to mixed GEOGRAPHY(ANY) is allowed.
case (gt1: GeographyType, gt2: GeographyType) if !gt1.isMixedSrid && gt2.isMixedSrid =>
true
// Casting from GEOGRAPHY to GEOMETRY with the same SRID is allowed.
case (geog: GeographyType, geom: GeometryType) if geog.srid == geom.srid =>
true
Expand Down Expand Up @@ -1232,6 +1238,7 @@ case class Cast(
case FloatType => castToFloat(from)
case LongType => castToLong(from)
case DoubleType => castToDouble(from)
case _: GeographyType => identity
case _: GeometryType => castToGeometry(from)
case array: ArrayType =>
castArray(from.asInstanceOf[ArrayType].elementType, array.elementType)
Expand Down Expand Up @@ -1341,6 +1348,7 @@ case class Cast(
case FloatType => castToFloatCode(from, ctx)
case LongType => castToLongCode(from, ctx)
case DoubleType => castToDoubleCode(from, ctx)
case _: GeographyType => (c, evPrim, _) => code"$evPrim = $c;"
case _: GeometryType => castToGeometryCode(from)

case array: ArrayType =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1491,6 +1491,27 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper {

// The following tests are confirming the behavior of casting between geospatial types.

test("Casting GeographyType to GeographyType") {
// Casting from fixed SRID GEOGRAPHY(<srid>) to mixed SRID GEOGRAPHY(ANY) is always allowed.
// Type casting is always safe in this direction, so no additional constraints are imposed.
// Casting from mixed SRID GEOGRAPHY(ANY) to fixed SRID GEOGRAPHY(<srid>) is not allowed.
// Type casting can be unsafe in this direction, because per-row SRID values may be different.

// Valid cast test cases.
val canCastTestCases: Seq[(DataType, DataType)] = Seq(
(GeographyType(4326), GeographyType("ANY"))
)
// Iterate over the test cases and verify casting.
canCastTestCases.foreach { case (fromType, toType) =>
// Cast can be performed from `fromType` to `toType`.
assert(Cast.canCast(fromType, toType))
assert(Cast.canAnsiCast(fromType, toType))
// Cast cannot be performed from `toType` to `fromType`.
assert(!Cast.canCast(toType, fromType))
assert(!Cast.canAnsiCast(toType, fromType))
}
}

test("Casting GeographyType to GeometryType") {
// Casting from GEOGRAPHY to GEOMETRY is only allowed if the SRIDs are the same.

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,35 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException
}


-- !query
SELECT hex(ST_AsBinary(CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOGRAPHY(ANY)))) AS result
-- !query analysis
Project [hex(st_asbinary(cast(st_geogfromwkb(0x0101000000000000000000F03F0000000000000040) as geography(any)))) AS result#x]
+- OneRowRelation


-- !query
SELECT CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040')::GEOGRAPHY(ANY) AS GEOGRAPHY(4326)) AS result
-- !query analysis
org.apache.spark.sql.catalyst.ExtendedAnalysisException
{
"errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION",
"sqlState" : "42K09",
"messageParameters" : {
"sqlExpr" : "\"CAST(CAST(st_geogfromwkb(X'0101000000000000000000F03F0000000000000040') AS GEOGRAPHY(ANY)) AS GEOGRAPHY(4326))\"",
"srcType" : "\"GEOGRAPHY(ANY)\"",
"targetType" : "\"GEOGRAPHY(4326)\""
},
"queryContext" : [ {
"objectType" : "",
"objectName" : "",
"startIndex" : 8,
"stopIndex" : 109,
"fragment" : "CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040')::GEOGRAPHY(ANY) AS GEOGRAPHY(4326))"
} ]
}


-- !query
SELECT hex(ST_AsBinary(CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOMETRY(4326)))) AS result
-- !query analysis
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,35 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException
}


-- !query
SELECT hex(ST_AsBinary(CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOGRAPHY(ANY)))) AS result
-- !query analysis
Project [hex(st_asbinary(cast(st_geogfromwkb(0x0101000000000000000000F03F0000000000000040) as geography(any)))) AS result#x]
+- OneRowRelation


-- !query
SELECT CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040')::GEOGRAPHY(ANY) AS GEOGRAPHY(4326)) AS result
-- !query analysis
org.apache.spark.sql.catalyst.ExtendedAnalysisException
{
"errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION",
"sqlState" : "42K09",
"messageParameters" : {
"sqlExpr" : "\"CAST(CAST(st_geogfromwkb(X'0101000000000000000000F03F0000000000000040') AS GEOGRAPHY(ANY)) AS GEOGRAPHY(4326))\"",
"srcType" : "\"GEOGRAPHY(ANY)\"",
"targetType" : "\"GEOGRAPHY(4326)\""
},
"queryContext" : [ {
"objectType" : "",
"objectName" : "",
"startIndex" : 8,
"stopIndex" : 109,
"fragment" : "CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040')::GEOGRAPHY(ANY) AS GEOGRAPHY(4326))"
} ]
}


-- !query
SELECT hex(ST_AsBinary(CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOMETRY(4326)))) AS result
-- !query analysis
Expand Down
5 changes: 5 additions & 0 deletions sql/core/src/test/resources/sql-tests/inputs/st-functions.sql
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,11 @@ INSERT INTO geodata VALUES
SELECT CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS STRING) AS result;
SELECT CAST(X'0101000000000000000000f03f0000000000000040' AS GEOMETRY(4326)) AS result;

-- Casting GEOGRAPHY(<srid>) to GEOGRAPHY(ANY) is allowed.
SELECT hex(ST_AsBinary(CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOGRAPHY(ANY)))) AS result;
-- Casting GEOGRAPHY(ANY) to GEOGRAPHY(<srid>) is not allowed.
SELECT CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040')::GEOGRAPHY(ANY) AS GEOGRAPHY(4326)) AS result;

-- Casting GEOGRAPHY to GEOMETRY is allowed only if SRIDs match.
SELECT hex(ST_AsBinary(CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOMETRY(4326)))) AS result;
-- Error handling: mismatched SRIDs.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -73,6 +73,38 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException
}


-- !query
SELECT hex(ST_AsBinary(CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOGRAPHY(ANY)))) AS result
-- !query schema
struct<result:string>
-- !query output
0101000000000000000000F03F0000000000000040


-- !query
SELECT CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040')::GEOGRAPHY(ANY) AS GEOGRAPHY(4326)) AS result
-- !query schema
struct<>
-- !query output
org.apache.spark.sql.catalyst.ExtendedAnalysisException
{
"errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION",
"sqlState" : "42K09",
"messageParameters" : {
"sqlExpr" : "\"CAST(CAST(st_geogfromwkb(X'0101000000000000000000F03F0000000000000040') AS GEOGRAPHY(ANY)) AS GEOGRAPHY(4326))\"",
"srcType" : "\"GEOGRAPHY(ANY)\"",
"targetType" : "\"GEOGRAPHY(4326)\""
},
"queryContext" : [ {
"objectType" : "",
"objectName" : "",
"startIndex" : 8,
"stopIndex" : 109,
"fragment" : "CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040')::GEOGRAPHY(ANY) AS GEOGRAPHY(4326))"
} ]
}


-- !query
SELECT hex(ST_AsBinary(CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOMETRY(4326)))) AS result
-- !query schema
Expand Down
32 changes: 32 additions & 0 deletions sql/core/src/test/resources/sql-tests/results/st-functions.sql.out
Original file line number Diff line number Diff line change
Expand Up @@ -73,6 +73,38 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException
}


-- !query
SELECT hex(ST_AsBinary(CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOGRAPHY(ANY)))) AS result
-- !query schema
struct<result:string>
-- !query output
0101000000000000000000F03F0000000000000040


-- !query
SELECT CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040')::GEOGRAPHY(ANY) AS GEOGRAPHY(4326)) AS result
-- !query schema
struct<>
-- !query output
org.apache.spark.sql.catalyst.ExtendedAnalysisException
{
"errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION",
"sqlState" : "42K09",
"messageParameters" : {
"sqlExpr" : "\"CAST(CAST(st_geogfromwkb(X'0101000000000000000000F03F0000000000000040') AS GEOGRAPHY(ANY)) AS GEOGRAPHY(4326))\"",
"srcType" : "\"GEOGRAPHY(ANY)\"",
"targetType" : "\"GEOGRAPHY(4326)\""
},
"queryContext" : [ {
"objectType" : "",
"objectName" : "",
"startIndex" : 8,
"stopIndex" : 109,
"fragment" : "CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040')::GEOGRAPHY(ANY) AS GEOGRAPHY(4326))"
} ]
}


-- !query
SELECT hex(ST_AsBinary(CAST(ST_GeogFromWKB(X'0101000000000000000000f03f0000000000000040') AS GEOMETRY(4326)))) AS result
-- !query schema
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ class STExpressionsSuite
// Private common constants used across several tests.
private final val defaultGeographySrid: Int = ExpressionDefaults.DEFAULT_GEOGRAPHY_SRID
private final val defaultGeographyType: DataType = GeographyType(defaultGeographySrid)
private final val mixedSridGeographyType: DataType = GeographyType("ANY")
private final val defaultGeometrySrid: Int = ExpressionDefaults.DEFAULT_GEOMETRY_SRID
private final val defaultGeometryType: DataType = GeometryType(defaultGeometrySrid)

Expand All @@ -39,6 +40,48 @@ class STExpressionsSuite
assert(sql(query).schema.fields.head.dataType.sameType(expectedDataType))
}

/** Geospatial type casting. */

test("Cast GEOGRAPHY(srid) to GEOGRAPHY(ANY)") {
// Test data: WKB representation of POINT(1 2).
val wkbString = "0101000000000000000000F03F0000000000000040"
val wkb = Hex.unhex(wkbString.getBytes())
val wkbLiteral = Literal.create(wkb, BinaryType)

// Construct the input GEOGRAPHY expression.
val geogExpr = ST_GeogFromWKB(wkbLiteral)
assert(geogExpr.dataType.sameType(defaultGeographyType))
checkEvaluation(ST_AsBinary(geogExpr), wkb)
// Cast the GEOGRAPHY with fixed SRID to GEOGRAPHY with mixed SRID.
val castExpr = Cast(geogExpr, mixedSridGeographyType)
assert(castExpr.dataType.sameType(mixedSridGeographyType))
checkEvaluation(ST_AsBinary(castExpr), wkb)

// Construct the input GEOGRAPHY SQL query, using WKB literal.
val geogQueryLit: String = s"ST_GeogFromWKB(X'$wkbString')"
assertType(s"SELECT $geogQueryLit", defaultGeographyType)
checkAnswer(sql(s"SELECT ST_AsBinary($geogQueryLit)"), Row(wkb))
// Cast the GEOGRAPHY with fixed SRID to GEOGRAPHY with mixed SRID.
val castQueryLit = s"$geogQueryLit::GEOGRAPHY(ANY)"
assertType(s"SELECT $castQueryLit", mixedSridGeographyType)
checkAnswer(sql(s"SELECT ST_AsBinary($castQueryLit)"), Row(wkb))

withTable("tbl") {
// Construct the test table with WKB.
sql(s"CREATE TABLE tbl (wkb BINARY)")
sql(s"INSERT INTO tbl VALUES (X'$wkbString')")

// Construct the input GEOGRAPHY SQL query, using WKB column.
val geogQueryCol: String = s"ST_GeogFromWKB(wkb)"
assertType(s"SELECT $geogQueryCol FROM tbl", defaultGeographyType)
checkAnswer(sql(s"SELECT ST_AsBinary($geogQueryCol) FROM tbl"), Row(wkb))
// Cast the GEOGRAPHY with fixed SRID to GEOGRAPHY with mixed SRID.
val castQueryCol = s"$geogQueryCol::GEOGRAPHY(ANY)"
assertType(s"SELECT $castQueryCol FROM tbl", mixedSridGeographyType)
checkAnswer(sql(s"SELECT ST_AsBinary($castQueryCol) FROM tbl"), Row(wkb))
}
}

/** ST reader/writer expressions. */

test("ST_AsBinary") {
Expand Down