You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2021/12/13 13:37:17 UTC

[GitHub] [iceberg] RussellSpitzer commented on issue #3731: Cannot use RewriteDataFiles on Hadoop Tables

RussellSpitzer commented on issue #3731:
URL: https://github.com/apache/iceberg/issues/3731#issuecomment-992483824


   There are a few approaches here. The key code is that the Spark implementation of the rewrite strategy uses 
   
   https://github.com/apache/iceberg/blob/551bda56db428037373efaae2bf8fc6e7b0be95a/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/actions/Spark3BinPackStrategy.java#L64-L68
   
   To read and write to the table, which assumes that the correct catalog identifier is included in the `name` of the table which has been passed to the action. We do this because we are using a Spark Reader and Writer to do the work here, so we need to know how to load the Table in a Spark context. When you pass in a table identifier that is loaded by an anonymous Iceberg catalog like ``` val tables = new HadoopTables(spark.sparkContext.hadoopConfiguration)``` it has to guess what the catalog name should be, and uses "default_catalog"
   
   To fix this we have a few options:
   
   1. Load the table via our helper method, here you can pass in the "catalog.database.name" of the Hadoop catalog https://github.com/apache/iceberg/blob/827b6c86108eec7b6de25f61022c7f8b5dda481c/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java#L628-L635
   2.  As you showed above, setting the default-iceberg catalog up, this way any identifiers that don't have a catalog will be checked there. There is no problem with this
   3. Load the table manually via Spark, basically doing *1.* on your own
       ```scala
        spark.sessionState.catalogManager.catalog("YourCatalogName").asInstanceOf[TableCatalog]
         .loadTable(tableName).asInstanceOf[org.apache.iceberg.spark.source.SparkTable]
         .table()
       ```


-- 
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-unsubscribe@iceberg.apache.org

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



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