tomtongue commented on issue #7317:
URL: https://github.com/apache/iceberg/issues/7317#issuecomment-1782418606

   Sorry for jumping in. I personally investigated the migrate query issue for 
GlueCatalog, so let me share my investigation result.
   
   ## Result
   Currently, it’s NOT possible to run `migrate` query for Spark/Hive tables in 
Glue Data Catalog. The reason of this is that GlueCatalog client doesn’t 
support renaming tables currently. 
   Let me elaborate that below. If I’m wrong, please correct me.
   
   ## Details
   When running the `migrate`  query for a Spark/Hive table in Glue Data 
Catalog, as described above, the `SparkSessionCatalog` configuration should be 
specified like `  .config("spark.sql.catalog.spark_catalog", 
"org.apache.iceberg.spark.SparkSessionCatalog")`.
   
   In this case, the source table in the `migrate` query like `table => 
'$db.$table'` is always set to `spark_catalog` (if other catalog is specified, 
the Spark application will fail).
   
   For this, in the current design of `migrate` , the code path always goes 
through 
[`SparkSessionCatalog.renameTable`](https://github.com/apache/iceberg/blob/apache-iceberg-1.4.1/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSessionCatalog.java#L293)
 because as its specification, the `migrate`  creates a staging table, renames 
the source table to keep the table as backup, and then migrate the source table 
to Iceberg. After the migration, the back table is dropped or not based on the 
`drop_backup` parameter. In the phase of renaming the source table to keep the 
backup table, the `SparkSessionCatalog.renameTable` is called. 
   
   The `SparkSessionCatalog.renameTable` can handle the IcebergCatalog to 
rename the table in GlueCatalog, the method basically checks the source table 
and if the source table is Iceberg, then calls `IcebergCatalog.renameTable` 
(GlueCatalogImpl is specified here, so the `renameTable` in GlueCatalogImpl 
will be used). However, in this case, the source table always belongs to 
`spark_catalog`, therefore the code path always goes to 
`getSessionCatalog().renameTable` as follows:
   
   
https://github.com/apache/iceberg/blob/apache-iceberg-1.4.1/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSessionCatalog.java#L293
   ```java
     @Override
     public void renameTable(Identifier from, Identifier to)
         throws NoSuchTableException, TableAlreadyExistsException {
       // rename is not supported by HadoopCatalog. to avoid 
UnsupportedOperationException for session
       // catalog tables,
       // check table existence first to ensure that the table belongs to the 
Iceberg catalog.
       if (icebergCatalog.tableExists(from)) {
         icebergCatalog.renameTable(from, to);
       } else {
         getSessionCatalog().renameTable(from, to);  // <= THIS PATH
       }
     }
   
   ```
   
   `getSessionCatalog().renameTable` calls Hive APIs for the table in Glue Data 
Catalog, so it fails due to renaming failure. 
   
   
   Here’s the detail of calling flow (in Iceberg 1.4.1 with Spark 3.5):
   1. 
https://github.com/apache/iceberg/blob/apache-iceberg-1.4.1/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/MigrateTableProcedure.java#L76
 -> Calls `MigrateTableSparkAction`
   2. 
https://github.com/apache/iceberg/blob/apache-iceberg-1.4.1/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateTableSparkAction.java#L118
 -> The actual migration impl
      1. `renameAndBackupSourceTable()` is called to keep the backup
      2. 
https://github.com/apache/iceberg/blob/apache-iceberg-1.4.1/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateTableSparkAction.java#L209
 -> `renameAndBackupSourceTable`. `destCatalog().renameTable(...)` will be 
called. But the `destCatalog()` is defined by `this.destCatalog = 
checkDestinationCatalog(sourceCatalog);` in the L66 in the same Class.
   3. 
https://github.com/apache/iceberg/blob/apache-iceberg-1.4.1/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSessionCatalog.java#L293
 -> The `destCatalog` is `SparkSessionCatalog` in step 2, so the 
`getSessionCatalog().renameTable` will be called.
   
   ## Resolution
   If the GlueCatalog renameTable can be used to keep the backup table, it’s 
possible to run the `migrate`. To resolve this, for example, it’s possible to 
add a new option to specify the destination catalog.


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

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