pvary commented on pull request #1505:
URL: https://github.com/apache/iceberg/pull/1505#issuecomment-700649623


   > > We need the build changes, because the spark sql uses Hive code to 
interact with hive tables.
   > 
   > That sounds like a bug to me. We want to ensure that Spark does not 
interact with Iceberg tables using Hive. It should use Iceberg directly through 
the v2 interface instead. We'll need to find out why Spark is trying to load 
the table incorrectly. What catalog were you using?
   
   I did the following 2 changes in the Iceberg code, and run the spark3 tests:
   - Remove the additional dependencies, and 
   - Set the default value for  to `false` and run the spark3 tests.
   
   One such test is `TestCreateTable.testCreateTable`. The drop table in the 
"after" fails with the exception below.
   
   It uses HiveCatalog. See: 
[code](https://github.com/apache/iceberg/blob/1772f4f27b8a12d3e89a7f65b8b600b717e1f09d/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java#L63)
   
   It tries to run `DROP TABLE IF EXISTS testhive.default.table` using 
`spark.sql`. My understanding is that this is spark code, and it tries to 
delegate it to the embedded `org.apache.hadoop.hive.ql.metadata.Table` class 
which tries to read the StorageHandler, and fails.
   
   This would mean that when Iceberg tables are using the HiveCatalog and Spark 
code tries to access them with SparkSQL, it will fail for several commands if 
we do not have `HiveIcebergStorageHandler` on the classpath.
   
   I am not sure we can change this part of Spark. I think,
   - We either have to add a note to the docs that an additional jar is needed 
if SparkSQL and HiveCatalog is used in conjunction, or 
   - We have to add the jars to our runtime jars.
   
   Your thoughts?
   
   The exception stack trace:
   ```
       java.lang.RuntimeException: 
org.apache.hadoop.hive.ql.metadata.HiveException: Error in loading storage 
handler.org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
           at 
org.apache.hadoop.hive.ql.metadata.Table.getStorageHandler(Table.java:297)
           at 
org.apache.spark.sql.hive.client.HiveClientImpl.convertHiveTableToCatalogTable(HiveClientImpl.scala:465)
           at 
org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$getTableOption$3(HiveClientImpl.scala:424)
           at scala.Option.map(Option.scala:230)
           at 
org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$getTableOption$1(HiveClientImpl.scala:424)
           at 
org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$withHiveState$1(HiveClientImpl.scala:294)
           at 
org.apache.spark.sql.hive.client.HiveClientImpl.liftedTree1$1(HiveClientImpl.scala:227)
           at 
org.apache.spark.sql.hive.client.HiveClientImpl.retryLocked(HiveClientImpl.scala:226)
           at 
org.apache.spark.sql.hive.client.HiveClientImpl.withHiveState(HiveClientImpl.scala:276)
           at 
org.apache.spark.sql.hive.client.HiveClientImpl.getTableOption(HiveClientImpl.scala:422)
           at 
org.apache.spark.sql.hive.client.HiveClient.getTable(HiveClient.scala:90)
           at 
org.apache.spark.sql.hive.client.HiveClient.getTable$(HiveClient.scala:89)
           at 
org.apache.spark.sql.hive.client.HiveClientImpl.getTable(HiveClientImpl.scala:90)
           at 
org.apache.spark.sql.hive.HiveExternalCatalog.getRawTable(HiveExternalCatalog.scala:120)
           at 
org.apache.spark.sql.hive.HiveExternalCatalog.$anonfun$getTable$1(HiveExternalCatalog.scala:719)
           at 
org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:99)
           at 
org.apache.spark.sql.hive.HiveExternalCatalog.getTable(HiveExternalCatalog.scala:719)
           at 
org.apache.spark.sql.catalyst.catalog.ExternalCatalogWithListener.getTable(ExternalCatalogWithListener.scala:138)
           at 
org.apache.spark.sql.catalyst.catalog.SessionCatalog.getTableMetadata(SessionCatalog.scala:446)
           at 
org.apache.spark.sql.execution.command.DropTableCommand.run(ddl.scala:226)
           at 
org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:70)
           at 
org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:68)
           at 
org.apache.spark.sql.execution.command.ExecutedCommandExec.executeCollect(commands.scala:79)
           at 
org.apache.spark.sql.Dataset.$anonfun$logicalPlan$1(Dataset.scala:229)
           at 
org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:3616)
           at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:100)
           at 
org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:160)
           at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:87)
           at 
org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:763)
           at 
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
           at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3614)
           at org.apache.spark.sql.Dataset.<init>(Dataset.scala:229)
           at org.apache.spark.sql.Dataset$.$anonfun$ofRows$2(Dataset.scala:100)
           at 
org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:763)
           at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:97)
           at 
org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:606)
           at 
org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:763)
           at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:601)
           at org.apache.iceberg.spark.SparkTestBase.sql(SparkTestBase.java:83)
           at 
org.apache.iceberg.spark.sql.TestCreateTable.dropTestTable(TestCreateTable.java:47)
   ```


----------------------------------------------------------------
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.

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