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

   ### Apache Iceberg version
   
   0.14.0 (latest release)
   
   ### Query engine
   
   Spark
   
   ### Please describe the bug 🐞
   
   If hive support is not enabled for SparkSession, Spark will use in-memory 
catalog for its built-in catalog. This leads to Iceberg's SparkSessionCatalog 
delegating requests to the in-memory catalog, which can be confusing since user 
expects requests to be handled by hive catalog. 
   For example, consider a hive metastore at thrift://localhost:9093 with 
namespaces:
   ```
   +---------+
   |namespace|
   +---------+
   |       db|
   |  default|
   +---------+
   ```
   and a table in namespace "db":
   ```
   +---------+---------+-----------+
   |namespace|tableName|isTemporary|
   +---------+---------+-----------+
   |       db|    table|      false|
   +---------+---------+-----------+
   ```
   The following steps will produce inconsistent results:
   1. Build a SparkSession without hive support:
   ```
   SparkSession spark = SparkSession
           .builder()
           .appName("SparkHiveTest")
           .master("local[*]")
           .config("spark.sql.extensions", 
"org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions")
           .config("spark.sql.catalog.spark_catalog", 
"org.apache.iceberg.spark.SparkSessionCatalog")
           .config("spark.sql.catalog.spark_catalog.type", "hive")
           .config("spark.sql.catalog.spark_catalog.uri", 
"thrift://localhost:9083")
           .config("spark.sql.catalog.spark_catalog.warehouse", 
"file:///tmp/spark-warehouse")
           .getOrCreate();
   ```
   2. List namespaces, which will be handled by spark's in-memory catalog, 
therefore not showing namespace "db":
   ```
   spark.sql("SHOW DATABASES").show();
   
   +---------+
   |namespace|
   +---------+
   |  default|
   +---------+
   ```
   3. Create namespace "db", which will be handled by spark's in-memory 
catalog, therefore succeeds:
   ```
   spark.sql("CREATE DATABASE db");
   ```
   4. Create table "db.table", which will be handled by iceberg's hive catalog, 
where "db.table" already exists, therefore fails:
   ```
   spark.sql("CREATE TABLE db.table (id bigint, data string) USING ICEBERG"); 
// fails: Table db.table already exists
   ```
   However, user expects step 4 to succeed, since "db" is a new empty namespace 
from user's perspective.
   
   To avoid such problems, we should add a check to ensure hive support for 
spark is enabled when initializing SparkSessionCatalog.
   


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