delta-io / delta

An open-source storage framework that enables building a Lakehouse architecture with compute engines including Spark, PrestoDB, Flink, Trino, and Hive and APIs
https://delta.io
Apache License 2.0
7.62k stars 1.71k forks source link

[BUG][Spark] UniForm table creation fails with "shadedForDelta.org.apache.iceberg.mr.hive.HiveIcebergSerDe not found" if Hive is Iceberg-enabled #3839

Open orthoxerox opened 2 weeks ago

orthoxerox commented 2 weeks ago

Bug

Which Delta project/connector is this regarding?

Describe the problem

When I create a UniForm table with Iceberg support, Delta Lake creates a table in the HMS that neither Spark nor Hive can interact with, I have to update the serde to org.apache.iceberg.mr.hive.HiveIcebergSerDe (unshaded) to delete it from Hive, Spark complains it can't read it either.

I have to run spark with --conf "spark.hadoop.iceberg.engine.hive.enabled=false" to make Delta UniForm work correctly.

Steps to reproduce

  1. Use Hive with iceberg.engine.hive.enabled set to true in hive-site.xml
  2. spark-shell --jars 'antlr4-runtime-4.9.3.jar,caffeine-2.9.3.jar,checker-qual-3.19.0.jar,delta-iceberg_2.13-3.2.1.jar,delta-spark_2.13-3.2.1.jar,delta-storage-3.2.1.jar,error_prone_annotations-2.10.0.jar,scala-collection-compat_2.13-2.1.1.jar' --conf "spark.sql.extensions=io.delta.sql.DeltaSparkSessionExtension" --conf "spark.sql.catalog.spark_catalog=org.apache.spark.sql.delta.catalog.DeltaCatalog"
  3. spark.sql("create table test_delta3(n bigint) using DELTA TBLPROPERTIES ('delta.enableIcebergCompatV2' = 'true', 'delta.universalFormat.enabledFormats' = 'iceberg')")

Observed results

24/11/02 11:00:23 ERROR log: error in initSerDe: java.lang.ClassNotFoundException Class shadedForDelta.org.apache.iceberg.mr.hive.HiveIcebergSerDe not found
java.lang.ClassNotFoundException: Class shadedForDelta.org.apache.iceberg.mr.hive.HiveIcebergSerDe not found
        at org.apache.hadoop.conf.Configuration.getClassByName(Configuration.java:2597)
        at org.apache.hadoop.hive.metastore.MetaStoreUtils.getDeserializer(MetaStoreUtils.java:447)
        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.getDeserializer(Table.java:263)
        at org.apache.hadoop.hive.ql.metadata.Table.getColsInternal(Table.java:641)
        at org.apache.hadoop.hive.ql.metadata.Table.getCols(Table.java:624)
        at org.apache.spark.sql.hive.client.HiveClientImpl.liftedTree2$1(HiveClientImpl.scala:455)
        at org.apache.spark.sql.hive.client.HiveClientImpl.org$apache$spark$sql$hive$client$HiveClientImpl$$convertHiveTableToCatalogTable(HiveClientImpl.scala:454)
        at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$getTableOption$3(HiveClientImpl.scala:441)
        at scala.Option.map(Option.scala:242)
        at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$getTableOption$1(HiveClientImpl.scala:441)
        at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$withHiveState$1(HiveClientImpl.scala:302)
        at org.apache.spark.sql.hive.client.HiveClientImpl.liftedTree1$1(HiveClientImpl.scala:233)
        at org.apache.spark.sql.hive.client.HiveClientImpl.retryLocked(HiveClientImpl.scala:232)
        at org.apache.spark.sql.hive.client.HiveClientImpl.withHiveState(HiveClientImpl.scala:282)
        at org.apache.spark.sql.hive.client.HiveClientImpl.getTableOption(HiveClientImpl.scala:439)
        at org.apache.spark.sql.hive.client.HiveClient.getTable(HiveClient.scala:98)
        at org.apache.spark.sql.hive.client.HiveClient.getTable$(HiveClient.scala:97)
        at org.apache.spark.sql.hive.client.HiveClientImpl.getTable(HiveClientImpl.scala:92)
        at org.apache.spark.sql.hive.HiveExternalCatalog.getRawTable(HiveExternalCatalog.scala:120)
        at org.apache.spark.sql.hive.HiveExternalCatalog.$anonfun$getTable$1(HiveExternalCatalog.scala:746)
        at org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:99)
        at org.apache.spark.sql.hive.HiveExternalCatalog.getTable(HiveExternalCatalog.scala:746)
        at org.apache.spark.sql.catalyst.catalog.ExternalCatalogWithListener.getTable(ExternalCatalogWithListener.scala:138)
        at org.apache.spark.sql.catalyst.catalog.SessionCatalog.getTableRawMetadata(SessionCatalog.scala:546)
        at org.apache.spark.sql.catalyst.catalog.SessionCatalog.getTableMetadata(SessionCatalog.scala:530)
        at org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog.loadTable(V2SessionCatalog.scala:75)
        at org.apache.spark.sql.connector.catalog.DelegatingCatalogExtension.loadTable(DelegatingCatalogExtension.java:73)
        at org.apache.spark.sql.delta.catalog.DeltaCatalog.super$loadTable(DeltaCatalog.scala:222)
        at org.apache.spark.sql.delta.catalog.DeltaCatalog.$anonfun$loadTable$1(DeltaCatalog.scala:222)
        at org.apache.spark.sql.delta.metering.DeltaLogging.recordFrameProfile(DeltaLogging.scala:168)
        at org.apache.spark.sql.delta.metering.DeltaLogging.recordFrameProfile$(DeltaLogging.scala:166)
        at org.apache.spark.sql.delta.catalog.DeltaCatalog.recordFrameProfile(DeltaCatalog.scala:66)
        at org.apache.spark.sql.delta.catalog.DeltaCatalog.loadTable(DeltaCatalog.scala:221)
        at org.apache.spark.sql.delta.catalog.DeltaCatalog.$anonfun$createDeltaTable$1(DeltaCatalog.scala:216)
        at org.apache.spark.sql.delta.metering.DeltaLogging.recordFrameProfile(DeltaLogging.scala:168)
        at org.apache.spark.sql.delta.metering.DeltaLogging.recordFrameProfile$(DeltaLogging.scala:166)
        at org.apache.spark.sql.delta.catalog.DeltaCatalog.recordFrameProfile(DeltaCatalog.scala:66)
        at org.apache.spark.sql.delta.catalog.DeltaCatalog.org$apache$spark$sql$delta$catalog$DeltaCatalog$$createDeltaTable(DeltaCatalog.scala:102)
        at org.apache.spark.sql.delta.catalog.DeltaCatalog.$anonfun$createTable$1(DeltaCatalog.scala:363)
        at org.apache.spark.sql.delta.metering.DeltaLogging.recordFrameProfile(DeltaLogging.scala:168)
        at org.apache.spark.sql.delta.metering.DeltaLogging.recordFrameProfile$(DeltaLogging.scala:166)
        at org.apache.spark.sql.delta.catalog.DeltaCatalog.recordFrameProfile(DeltaCatalog.scala:66)
        at org.apache.spark.sql.delta.catalog.DeltaCatalog.createTable(DeltaCatalog.scala:343)
        at org.apache.spark.sql.delta.catalog.DeltaCatalog.createTable(DeltaCatalog.scala:334)
        at org.apache.spark.sql.execution.datasources.v2.CreateTableExec.run(CreateTableExec.scala:44)
        at org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result$lzycompute(V2CommandExec.scala:43)
        at org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result(V2CommandExec.scala:43)
        at org.apache.spark.sql.execution.datasources.v2.V2CommandExec.executeCollect(V2CommandExec.scala:49)
        at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.$anonfun$applyOrElse$1(QueryExecution.scala:107)
        at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$6(SQLExecution.scala:125)
        at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:201)
        at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:108)
        at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:900)
        at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:66)
        at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:107)
        at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:98)
        at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:461)
        at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:76)
        at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:461)
        at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:32)
        at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267)
        at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263)
        at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:32)
        at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:32)
        at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:437)
        at org.apache.spark.sql.execution.QueryExecution.eagerlyExecuteCommands(QueryExecution.scala:98)
        at org.apache.spark.sql.execution.QueryExecution.commandExecuted$lzycompute(QueryExecution.scala:85)
        at org.apache.spark.sql.execution.QueryExecution.commandExecuted(QueryExecution.scala:83)
        at org.apache.spark.sql.Dataset.<init>(Dataset.scala:220)
        at org.apache.spark.sql.Dataset$.$anonfun$ofRows$2(Dataset.scala:100)
        at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:900)
        at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:97)
        at org.apache.spark.sql.SparkSession.$anonfun$sql$4(SparkSession.scala:691)
        at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:900)
        at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:682)
        at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:713)
        at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:744)
        at $line11.$read$$iw.<init>(<console>:1)
        at $line11.$read.<init>(<console>:15)
        at $line11.$read$.<clinit>(<console>:1)
        at $line11.$eval$.$print$lzycompute(<synthetic>:6)
        at $line11.$eval$.$print(<synthetic>:5)
        at $line11.$eval.$print(<synthetic>)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:498)
        at scala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:670)
        at scala.tools.nsc.interpreter.IMain$Request.loadAndRun(IMain.scala:1020)
        at scala.tools.nsc.interpreter.IMain.$anonfun$doInterpret$1(IMain.scala:506)
        at scala.reflect.internal.util.ScalaClassLoader.asContext(ScalaClassLoader.scala:36)
        at scala.reflect.internal.util.ScalaClassLoader.asContext$(ScalaClassLoader.scala:116)
        at scala.reflect.internal.util.AbstractFileClassLoader.asContext(AbstractFileClassLoader.scala:43)
        at scala.tools.nsc.interpreter.IMain.loadAndRunReq$1(IMain.scala:505)
        at scala.tools.nsc.interpreter.IMain.$anonfun$doInterpret$3(IMain.scala:519)
        at scala.tools.nsc.interpreter.IMain.doInterpret(IMain.scala:519)
        at scala.tools.nsc.interpreter.IMain.interpret(IMain.scala:503)
        at scala.tools.nsc.interpreter.IMain.interpret(IMain.scala:501)
        at scala.tools.nsc.interpreter.shell.ILoop.loop$1(ILoop.scala:878)
        at scala.tools.nsc.interpreter.shell.ILoop.interpretStartingWith(ILoop.scala:906)
        at scala.tools.nsc.interpreter.shell.ILoop.command(ILoop.scala:433)
        at scala.tools.nsc.interpreter.shell.ILoop.processLine(ILoop.scala:440)
        at scala.tools.nsc.interpreter.shell.ILoop.loop(ILoop.scala:458)
        at scala.tools.nsc.interpreter.shell.ILoop.run(ILoop.scala:968)
        at org.apache.spark.repl.Main$.doMain(Main.scala:84)
        at org.apache.spark.repl.Main$.main(Main.scala:59)
        at org.apache.spark.repl.Main.main(Main.scala)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:498)
        at org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)
        at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:1029)
        at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:194)
        at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:217)
        at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:91)
        at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1120)
        at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1129)
        at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)

Expected results

I expected Delta Lake to create the table.

Further details

Environment information

Willingness to contribute

The Delta Lake Community encourages bug fix contributions. Would you or another member of your organization be willing to contribute a fix for this bug to the Delta Lake code base?