liuj84 opened a new issue, #10401:
URL: https://github.com/apache/iceberg/issues/10401

   ### Feature Request / Improvement
   
   Iceberg does not support loading different versions of hive-metastore jars 
at runtime. According to the [Spark SQL 
documentation](https://spark.apache.org/docs/latest/sql-data-sources-hive-tables.html#interacting-with-different-versions-of-hive-metastore),
  Spark SQL can query different versions of Hive metastores using the 
"spark.sql.hive.metastore.version" and "spark.sql.hive.metastore.jars" 
configurations. Spark SQL can load Hive Metastore jars from either a Maven 
repository or a specified file path.
   
   When executing SQL on the Hive catalog, Spark creates an isolated HiveClient 
using 
[IsolatedClientLoader](https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala#L238).
 This loader is responsible for loading an isolated version of 
HiveMetastoreClient from external jars specified by the 
spark.sql.hive.metastore.* configurations.
   
   However, the Iceberg catalog does not utilize the 
spark.sql.hive.metastore.version setting. Instead, when creating a 
HiveMetaStoreClient, the Iceberg code calls HiveClientPool.newClient() 
([source](https://github.com/apache/iceberg/blob/main/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveClientPool.java#L60)),
 which bypasses the Spark IsolatedClientLoader and loads the 
HiveMetastoreClient from the default classloader. As a result, only the Hive 
Metastore jar present in the classpath is loaded, ignoring the 
spark.sql.hive.metastore.version setting.
   
   Context
   I am writing a Spark job to query Iceberg data using the following code:
   ``` java
   SparkSession spark = SparkSession.builder()
       .master("local[*]")
       .appName("Spark app simple")
       .config("spark.sql.hive.metastore.version", "3.1.3")  <- this doesn't 
take effect for iceberg catalog
       .config("spark.sql.hive.metastore.jars", "maven")
       .config("spark.sql.extensions", 
"org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions")
       .config("spark.hadoop.fs.s3a.endpoint", "localhost:9021")
       .config("spark.hadoop.fs.s3a.access.key", "test")
       .config("spark.hadoop.fs.s3a.secret.key", "xxx")
       .config("spark.hadoop.fs.s3a.path.style.access", "true")
       .config("spark.sql.catalog.iceberg", 
"org.apache.iceberg.spark.SparkCatalog")
       .config("spark.sql.catalog.iceberg.uri", "thrift://localhost:9090")
       .enableHiveSupport()
       .getOrCreate();
   
   spark.sql("CREATE TABLE IF NOT EXISTS iceberg.test.users (name VARCHAR(255), 
age INT)");
   
   ```
   Questions:
   Can Iceberg support loading different Hive versions at runtime similar to 
Spark SQL? Alternatively, are there existing solutions for this? Currently, the 
only way to use my desired Hive Metastore version is to load the jar in my 
classpath, which is not ideal.
   
   ### Query engine
   
   Hive


-- 
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: issues-unsubscr...@iceberg.apache.org.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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

Reply via email to