yaooqinn opened a new pull request, #47652:
URL: https://github.com/apache/spark/pull/47652

   ### What changes were proposed in this pull request?
   <!--
   Please clarify what changes you are proposing. The purpose of this section 
is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster 
reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class 
hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other 
DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   
   This PR marks the variant type as hive incompatible data type to avoid 
calling saveTableIntoHive twice
   
   
   ```
   spark-sql (default)> create table T using parquet as select parse_json('1') 
as o;
   24/08/07 17:53:35 WARN HiveExternalCatalog: Could not persist 
`spark_catalog`.`default`.`t` in a Hive compatible way. Persisting it into Hive 
metastore in Spark SQL specific format.
   org.apache.hadoop.hive.ql.metadata.HiveException: 
java.lang.IllegalArgumentException: Error: type expected at the position 0 of 
'variant' but 'variant' is found.
        at org.apache.hadoop.hive.ql.metadata.Hive.createTable(Hive.java:869)
        at org.apache.hadoop.hive.ql.metadata.Hive.createTable(Hive.java:874)
        at 
org.apache.spark.sql.hive.client.Shim_v2_0.createTable(HiveShim.scala:918)
        at 
org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$createTable$1(HiveClientImpl.scala:568)
        at 
scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.scala:18)
        at 
org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$withHiveState$1(HiveClientImpl.scala:298)
        at 
org.apache.spark.sql.hive.client.HiveClientImpl.liftedTree1$1(HiveClientImpl.scala:229)
        at 
org.apache.spark.sql.hive.client.HiveClientImpl.retryLocked(HiveClientImpl.scala:228)
        at 
org.apache.spark.sql.hive.client.HiveClientImpl.withHiveState(HiveClientImpl.scala:278)
        at 
org.apache.spark.sql.hive.client.HiveClientImpl.createTable(HiveClientImpl.scala:566)
        at 
org.apache.spark.sql.hive.HiveExternalCatalog.saveTableIntoHive(HiveExternalCatalog.scala:503)
        at 
org.apache.spark.sql.hive.HiveExternalCatalog.createDataSourceTable(HiveExternalCatalog.scala:392)
        at 
org.apache.spark.sql.hive.HiveExternalCatalog.$anonfun$createTable$1(HiveExternalCatalog.scala:266)
        at 
scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.scala:18)
        at 
org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:100)
        at 
org.apache.spark.sql.hive.HiveExternalCatalog.createTable(HiveExternalCatalog.scala:213)
        at 
org.apache.spark.sql.catalyst.catalog.ExternalCatalogWithListener.createTable(ExternalCatalogWithListener.scala:94)
        at 
org.apache.spark.sql.catalyst.catalog.SessionCatalog.createTable(SessionCatalog.scala:410)
        at 
org.apache.spark.sql.execution.command.CreateDataSourceTableAsSelectCommand.run(createDataSourceTables.scala:196)
        at 
org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:77)
        at 
org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:75)
        at 
org.apache.spark.sql.execution.command.ExecutedCommandExec.executeCollect(commands.scala:86)
        at 
org.apache.spark.sql.execution.QueryExecution.$anonfun$eagerlyExecuteCommands$2(QueryExecution.scala:126)
        at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId0$6(SQLExecution.scala:155)
        at 
org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:267)
        at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId0$1(SQLExecution.scala:118)
        at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:923)
        at 
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId0(SQLExecution.scala:74)
        at 
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:222)
        at 
org.apache.spark.sql.execution.QueryExecution.$anonfun$eagerlyExecuteCommands$1(QueryExecution.scala:126)
        at 
org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:608)
        at 
org.apache.spark.sql.execution.QueryExecution.org$apache$spark$sql$execution$QueryExecution$$eagerlyExecute$1(QueryExecution.scala:125)
        at 
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$3.applyOrElse(QueryExecution.scala:140)
        at 
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$3.applyOrElse(QueryExecution.scala:135)
        at 
org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:470)
        at 
org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:84)
        at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:470)
        at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:37)
        at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:330)
        at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:326)
        at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:37)
        at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:37)
        at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:446)
        at 
org.apache.spark.sql.execution.QueryExecution.eagerlyExecuteCommands(QueryExecution.scala:135)
        at 
org.apache.spark.sql.execution.QueryExecution.commandExecuted$lzycompute(QueryExecution.scala:100)
        at 
org.apache.spark.sql.execution.QueryExecution.commandExecuted(QueryExecution.scala:98)
        at org.apache.spark.sql.Dataset.<init>(Dataset.scala:240)
        at org.apache.spark.sql.Dataset$.$anonfun$ofRows$3(Dataset.scala:119)
        at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:923)
        at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:115)
        at 
org.apache.spark.sql.SparkSession.$anonfun$sql$4(SparkSession.scala:713)
        at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:923)
        at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:704)
        at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:735)
        at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:766)
        at 
org.apache.spark.sql.hive.thriftserver.SparkSQLDriver.run(SparkSQLDriver.scala:69)
        at 
org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processCmd(SparkSQLCLIDriver.scala:461)
        at 
org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.$anonfun$processLine$1(SparkSQLCLIDriver.scala:579)
        at 
org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.$anonfun$processLine$1$adapted(SparkSQLCLIDriver.scala:573)
        at scala.collection.IterableOnceOps.foreach(IterableOnce.scala:619)
        at scala.collection.IterableOnceOps.foreach$(IterableOnce.scala:617)
        at scala.collection.AbstractIterable.foreach(Iterable.scala:935)
        at 
org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:573)
        at 
org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:288)
        at 
org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)
        at 
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)
        at 
java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.base/java.lang.reflect.Method.invoke(Method.java:569)
        at 
org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)
        at 
org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:1025)
        at 
org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:199)
        at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:222)
        at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:96)
        at 
org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1129)
        at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1138)
        at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
   Caused by: java.lang.IllegalArgumentException: Error: type expected at the 
position 0 of 'variant' but 'variant' is found.
        at 
org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils$TypeInfoParser.expect(TypeInfoUtils.java:372)
        at 
org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils$TypeInfoParser.expect(TypeInfoUtils.java:355)
        at 
org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils$TypeInfoParser.parseType(TypeInfoUtils.java:416)
        at 
org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils$TypeInfoParser.parseTypeInfos(TypeInfoUtils.java:329)
        at 
org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils.getTypeInfosFromTypeString(TypeInfoUtils.java:814)
        at 
org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe.initialize(ParquetHiveSerDe.java:110)
        at 
org.apache.hadoop.hive.serde2.AbstractSerDe.initialize(AbstractSerDe.java:54)
        at 
org.apache.hadoop.hive.serde2.SerDeUtils.initializeSerDe(SerDeUtils.java:533)
        at 
org.apache.hadoop.hive.metastore.MetaStoreUtils.getDeserializer(MetaStoreUtils.java:453)
        at 
org.apache.hadoop.hive.metastore.MetaStoreUtils.getDeserializer(MetaStoreUtils.java:440)
        at 
org.apache.hadoop.hive.ql.metadata.Table.getDeserializerFromMetaStore(Table.java:281)
        at 
org.apache.hadoop.hive.ql.metadata.Table.checkValidity(Table.java:199)
        at org.apache.hadoop.hive.ql.metadata.Hive.createTable(Hive.java:842)
        ... 76 more
   Time taken: 1.014 seconds
   ``` 
   
   ### Why are the changes needed?
   
   trivial bugfix 
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as 
the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes 
- provide the console output, description and/or an example to show the 
behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to 
the released Spark versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   no
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some 
test cases that check the changes thoroughly including negative and positive 
cases if possible.
   If it was tested in a way different from regular unit tests, please clarify 
how you tested step by step, ideally copy and paste-able, so that other 
reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why 
it was difficult to add.
   If benchmark tests were added, please run the benchmarks in GitHub Actions 
for the consistent environment, and the instructions could accord to: 
https://spark.apache.org/developer-tools.html#github-workflow-benchmarks.
   -->
   
   manually
   
   ```
   spark-sql (default)> create table T using parquet as select parse_json('1') 
as o;
   24/08/07 18:32:21 WARN HiveExternalCatalog: Hive incompatible types found: 
variant. Persisting data source table `spark_catalog`.`default`.`t` into Hive 
metastore in Spark SQL specific format, which is NOT compatible with Hive.
   Time taken: 4.205 seconds
   ```
   ### Was this patch authored or co-authored using generative AI tooling?
   <!--
   If generative AI tooling has been used in the process of authoring this 
patch, please include the
   phrase: 'Generated-by: ' followed by the name of the tool and its version.
   If no, write 'No'.
   Please refer to the [ASF Generative Tooling 
Guidance](https://www.apache.org/legal/generative-tooling.html) for details.
   -->no
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to