RussellSpitzer opened a new issue #2374:
URL: https://github.com/apache/iceberg/issues/2374


   Talking with @szehon-ho we found a few issues with the drop pathway for 
SparkSessionCatalog
   
   Normally when a drop table statement is parsed it goes through the following 
steps
   ```
   Parse DropTableStatement Object
   
   // Attempt to Resolve as non session Catalog
   Apply ResolveCatalog
     If the catalog of the resolved drop command is the session catalog 
        do nothing
      else
         Create DropTable plan (eventually hits V2CatalogResolution Rules and 
calls V2 Catalog.dropTable
    
    // Attempt to resolve as session Catalog
    Apply ResolveSessionCatalog
      Create DropTableCommand plan
      Calls spark.sessionState.catalog.dropTable(table)
    ```
    
    The problem with the pathway here is that the spark.sessionState.catalog 
will always invoke the underlying delegate's dropTable method. This is bad for 
a few reasons
    
    1. Treats the table as a HiveTable when deleting and will ignore Iceberg 
specific properties. This means files that are not in the default location for 
the table will be ignored. 
    2. The Iceberg Catalog never sees the drop operation so the Cache there is 
never cleared
    3. If the Delegate catalog is using a different HMS (or catalog) than the 
Iceberg Catalog, a completely different table could be dropped or you get "no 
table found" when attempting to drop the table.
    
    
    
    A quick repo for Delegate Catalog is Different than Iceberg Catalog
    
    Create a session where spark is using Derby and Iceberg is using HMS
    ```bash
    ./bin/spark-shell --conf 
spark.sql.catalog.spark_catalog=org.apache.iceberg.spark.SparkSessionCatalog 
--conf spark.sql.catalog.spark_catalog.type=hive --conf 
spark.sql.catalog.spark_catalog.uri=thrift://localhost:9083
    ```
    
    ```scala
    scala> spark.sql("CREATE TABLE cold (ice int) USING iceberg LOCATION 
'file:///Users/russellspitzer/cold'")
     scala> spark.sql("INSERT INTO cold VALUES (1)")
     scala> spark.sql("SELECT * from cold").show
   +---+
   |ice|
   +---+
   |  1|
   +---+
   scala> spark.sql("DROP TABLE cold")
   org.apache.spark.sql.AnalysisException: Table or view not found: cold;
   ```
   
   We also have an issue where we always delegate "showTables" which also is a 
problem if the delegate catalog is not the same underlying store that the 
SparkSessionCatalog is using.


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