[ 
https://issues.apache.org/jira/browse/SPARK-40988?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17632073#comment-17632073
 ] 

Ranga Reddy commented on SPARK-40988:
-------------------------------------

In Spark 3.4, if we run the following code we can see the *CAST_INVALID_INPUT* 
exception.
{code:java}
spark.sql(s"""INSERT INTO $tableName partition (age=\"test_age\") VALUES (2, 
'Nishanth')"""){code}
*Exception:*
{code:java}
[CAST_INVALID_INPUT] The value 'AGE_34' of the type "STRING" cannot be cast to 
"INT" because it is malformed. Correct the value as per the syntax, or change 
its target type. Use `try_cast` to tolerate malformed input and return NULL 
instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this 
error.
== SQL(line 1, position 1) ==
INSERT INTO TABLE partition_table PARTITION(age="AGE_34") VALUES (1, 'ABC')
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^org.apache.spark.SparkNumberFormatException:
 [CAST_INVALID_INPUT] The value 'AGE_34' of the type "STRING" cannot be cast to 
"INT" because it is malformed. Correct the value as per the syntax, or change 
its target type. Use `try_cast` to tolerate malformed input and return NULL 
instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this 
error.
== SQL(line 1, position 1) ==
INSERT INTO TABLE partition_table PARTITION(age="AGE_34") VALUES (1, 'ABC')
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^    at 
org.apache.spark.sql.errors.QueryExecutionErrors$.invalidInputInCastToNumberError(QueryExecutionErrors.scala:161)
    at 
org.apache.spark.sql.catalyst.util.UTF8StringUtils$.withException(UTF8StringUtils.scala:51)
    at 
org.apache.spark.sql.catalyst.util.UTF8StringUtils$.toIntExact(UTF8StringUtils.scala:34)
    at 
org.apache.spark.sql.catalyst.expressions.Cast.$anonfun$castToInt$2(Cast.scala:927)
    at 
org.apache.spark.sql.catalyst.expressions.Cast.$anonfun$castToInt$2$adapted(Cast.scala:927)
    at org.apache.spark.sql.catalyst.expressions.Cast.buildCast(Cast.scala:588)
    at 
org.apache.spark.sql.catalyst.expressions.Cast.$anonfun$castToInt$1(Cast.scala:927)
    at 
org.apache.spark.sql.catalyst.expressions.Cast.nullSafeEval(Cast.scala:1285)
    at 
org.apache.spark.sql.catalyst.expressions.UnaryExpression.eval(Expression.scala:526)
    at 
org.apache.spark.sql.catalyst.expressions.UnaryExpression.eval(Expression.scala:522)
    at 
org.apache.spark.sql.util.PartitioningUtils$.normalizePartitionStringValue(PartitioningUtils.scala:56)
    at 
org.apache.spark.sql.util.PartitioningUtils$.$anonfun$normalizePartitionSpec$1(PartitioningUtils.scala:100)
    at 
scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:286)
    at scala.collection.Iterator.foreach(Iterator.scala:943)
    at scala.collection.Iterator.foreach$(Iterator.scala:943)
    at scala.collection.AbstractIterator.foreach(Iterator.scala:1431)
    at scala.collection.IterableLike.foreach(IterableLike.scala:74)
    at scala.collection.IterableLike.foreach$(IterableLike.scala:73)
    at scala.collection.AbstractIterable.foreach(Iterable.scala:56)
    at scala.collection.TraversableLike.map(TraversableLike.scala:286)
    at scala.collection.TraversableLike.map$(TraversableLike.scala:279)
    at scala.collection.AbstractTraversable.map(Traversable.scala:108)
    at 
org.apache.spark.sql.util.PartitioningUtils$.normalizePartitionSpec(PartitioningUtils.scala:76)
    at 
org.apache.spark.sql.execution.datasources.PreprocessTableInsertion$.org$apache$spark$sql$execution$datasources$PreprocessTableInsertion$$preprocess(rules.scala:382)
    at 
org.apache.spark.sql.execution.datasources.PreprocessTableInsertion$$anonfun$apply$3.applyOrElse(rules.scala:426)
    at 
org.apache.spark.sql.execution.datasources.PreprocessTableInsertion$$anonfun$apply$3.applyOrElse(rules.scala:420)
    at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsDownWithPruning$2(AnalysisHelper.scala:170)
    at 
org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:104)
    at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsDownWithPruning$1(AnalysisHelper.scala:170)
    at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.allowInvokingTransformsInAnalyzer(AnalysisHelper.scala:323)
    at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsDownWithPruning(AnalysisHelper.scala:168)
    at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsDownWithPruning$(AnalysisHelper.scala:164)
    at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperatorsDownWithPruning(LogicalPlan.scala:30)
    at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsWithPruning(AnalysisHelper.scala:99)
    at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsWithPruning$(AnalysisHelper.scala:96)
    at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperatorsWithPruning(LogicalPlan.scala:30)
    at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperators(AnalysisHelper.scala:76)
    at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperators$(AnalysisHelper.scala:75)
    at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperators(LogicalPlan.scala:30)
    at 
org.apache.spark.sql.execution.datasources.PreprocessTableInsertion$.apply(rules.scala:420)
    at 
org.apache.spark.sql.execution.datasources.PreprocessTableInsertion$.apply(rules.scala:374)
    at 
org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:211)
    at 
scala.collection.LinearSeqOptimized.foldLeft(LinearSeqOptimized.scala:126)
    at 
scala.collection.LinearSeqOptimized.foldLeft$(LinearSeqOptimized.scala:122)
    at scala.collection.immutable.List.foldLeft(List.scala:91)
    at 
org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:208)
    at 
org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:200)
    at scala.collection.immutable.List.foreach(List.scala:431)
    at 
org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:200)
    at 
org.apache.spark.sql.catalyst.analysis.Analyzer.org$apache$spark$sql$catalyst$analysis$Analyzer$$executeSameContext(Analyzer.scala:228)
    at 
org.apache.spark.sql.catalyst.analysis.Analyzer.$anonfun$execute$1(Analyzer.scala:224)
    at 
org.apache.spark.sql.catalyst.analysis.AnalysisContext$.withNewAnalysisContext(Analyzer.scala:173)
    at 
org.apache.spark.sql.catalyst.analysis.Analyzer.execute(Analyzer.scala:224)
    at 
org.apache.spark.sql.catalyst.analysis.Analyzer.execute(Analyzer.scala:188)
    at 
org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:179)
    at 
org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:88)
    at 
org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:179)
    at 
org.apache.spark.sql.catalyst.analysis.Analyzer.$anonfun$executeAndCheck$1(Analyzer.scala:209)
    at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.markInAnalyzer(AnalysisHelper.scala:330)
    at 
org.apache.spark.sql.catalyst.analysis.Analyzer.executeAndCheck(Analyzer.scala:208)
    at 
org.apache.spark.sql.hive.test.TestHiveQueryExecution.$anonfun$analyzed$1(TestHive.scala:624)
    at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:779)
    at 
org.apache.spark.sql.hive.test.TestHiveQueryExecution.analyzed$lzycompute(TestHive.scala:600)
    at 
org.apache.spark.sql.hive.test.TestHiveQueryExecution.analyzed(TestHive.scala:600)
    at 
org.apache.spark.sql.execution.QueryExecution.assertAnalyzed(QueryExecution.scala:66)
    at org.apache.spark.sql.Dataset$.$anonfun$ofRows$1(Dataset.scala:90)
    at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:779)
    at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:88)
    at 
org.apache.spark.sql.hive.test.TestHiveSparkSession.$anonfun$sql$1(TestHive.scala:240)
    at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:779)
    at 
org.apache.spark.sql.hive.test.TestHiveSparkSession.sql(TestHive.scala:238)
    at 
org.apache.spark.sql.test.SQLTestUtilsBase.$anonfun$sql$1(SQLTestUtils.scala:232)
    at 
org.apache.spark.sql.hive.InsertSuite.$anonfun$new$146(InsertSuite.scala:930)
    at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
    at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1491)
    at 
org.apache.spark.sql.test.SQLTestUtilsBase.withTable(SQLTestUtils.scala:306)
    at 
org.apache.spark.sql.test.SQLTestUtilsBase.withTable$(SQLTestUtils.scala:304)
    at org.apache.spark.sql.hive.InsertSuite.withTable(InsertSuite.scala:41)
    at 
org.apache.spark.sql.hive.InsertSuite.$anonfun$new$145(InsertSuite.scala:921)
    at 
org.apache.spark.sql.catalyst.plans.SQLHelper.withSQLConf(SQLHelper.scala:54)
    at 
org.apache.spark.sql.catalyst.plans.SQLHelper.withSQLConf$(SQLHelper.scala:38){code}

> Spark3 partition column value is not validated with user provided schema.
> -------------------------------------------------------------------------
>
>                 Key: SPARK-40988
>                 URL: https://issues.apache.org/jira/browse/SPARK-40988
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 3.0.0, 3.1.0, 3.2.0, 3.3.0
>            Reporter: Ranga Reddy
>            Priority: Major
>
> Spark3 has not validated the Partition Column type while inserting the data 
> but on the Hive side exception is thrown while inserting different type 
> values.
> *Spark Code:*
>  
> {code:java}
> scala> val tableName="test_partition_table"
> tableName: String = test_partition_table
> scala>scala> spark.sql(s"DROP TABLE IF EXISTS $tableName")
> res0: org.apache.spark.sql.DataFrame = []
> scala> spark.sql(s"CREATE EXTERNAL TABLE $tableName ( id INT, name STRING ) 
> PARTITIONED BY (age INT) LOCATION 'file:/tmp/spark-warehouse/$tableName'")
> res1: org.apache.spark.sql.DataFrame = []
> scala> spark.sql("SHOW tables").show(truncate=false)
> +---------+---------------------+-----------+
> |namespace|tableName            |isTemporary|
> +---------+---------------------+-----------+
> |default  |test_partition_table |false      |
> +---------+---------------------+-----------+
> scala> spark.sql("SET spark.sql.sources.validatePartitionColumns").show(50, 
> false)
> +------------------------------------------+-----+
> |key                                       |value|
> +------------------------------------------+-----+
> |spark.sql.sources.validatePartitionColumns|true |
> +------------------------------------------+-----+
> scala> spark.sql(s"""INSERT INTO $tableName partition (age=25) VALUES (1, 
> 'Ranga')""")
> res4: org.apache.spark.sql.DataFrame = []scala> spark.sql(s"show partitions 
> $tableName").show(50, false)
> +---------+
> |partition|
> +---------+
> |age=25   |
> +---------+
> scala> spark.sql(s"select * from $tableName").show(50, false)
> +---+-----+---+
> |id |name |age|
> +---+-----+---+
> |1  |Ranga|25 |
> +---+-----+---+
> scala> spark.sql(s"""INSERT INTO $tableName partition (age=\"test_age\") 
> VALUES (2, 'Nishanth')""")
> res7: org.apache.spark.sql.DataFrame = []scala> spark.sql(s"show partitions 
> $tableName").show(50, false)
> +------------+
> |partition   |
> +------------+
> |age=25      |
> |age=test_age|
> +------------+
> scala> spark.sql(s"select * from $tableName").show(50, false)
> +---+--------+----+
> |id |name    |age |
> +---+--------+----+
> |1  |Ranga   |25  |
> |2  |Nishanth|null|
> +---+--------+----+ {code}
> *Hive Code:*
>  
>  
> {code:java}
> > INSERT INTO test_partition_table partition (age="test_age2") VALUES (3, 
> > 'Nishanth');
> Error: Error while compiling statement: FAILED: SemanticException [Error 
> 10248]: Cannot add partition column age of type string as it cannot be 
> converted to type int (state=42000,code=10248){code}
>  
> *Expected Result:*
> When *spark.sql.sources.validatePartitionColumns=true* it needs to be 
> validated the datatype value and exception needs to be thrown if we provide 
> wrong data type value.
> *Reference:*
> [https://spark.apache.org/docs/3.3.1/sql-migration-guide.html#data-sources]



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org

Reply via email to