xiaoyuyao opened a new issue, #12737: URL: https://github.com/apache/iceberg/issues/12737
### Apache Iceberg version 1.8.1 (latest release) ### Query engine Spark ### Please describe the bug 🐞 The issue has been reported before in [thread1](https://github.com/apache/iceberg/issues/8377#issuecomment-1734973961) and [thread2](https://github.com/apache/iceberg/issues/8377#issuecomment-1691790773) A quick local repro step: `bin/spark-shell --packages org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:1.8.1,org.apache.iceberg:iceberg-hive-metastore:1.4.2 \ --conf spark.sql.extensions=org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions \ --conf spark.sql.catalog.spark_catalog=org.apache.iceberg.spark.SparkSessionCatalog scala> val df = Seq( | (1, "Alice"), | (2, "Bob") | ).toDF("id", "name") df: org.apache.spark.sql.DataFrame = [id: int, name: string] scala> df.write.format("iceberg").mode("append").save("/tmp/iceberg_test1") org.apache.spark.sql.catalyst.analysis.NoSuchTableException: [TABLE_OR_VIEW_NOT_FOUND] The table or view `/tmp`.iceberg_test1 cannot be found. Verify the spelling and correctness of the schema and catalog. If you did not qualify the name with a schema, verify the current_schema() output, or qualify the name with the correct schema and catalog. To tolerate the error on drop use DROP VIEW IF EXISTS or DROP TABLE IF EXISTS. at org.apache.iceberg.spark.SparkCatalog.loadTable(SparkCatalog.java:171) at org.apache.spark.sql.DataFrameWriter.saveInternal(DataFrameWriter.scala:295) at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:240) ` After checking the iceberg SparkCatalog, I found the cause of the failure is spark dataframe writer attempts to load the table first even with "overwrite"/"append" mode. In the case of table does not exist, iceberg SparkCatalog always throws NoSuchTableException. The expectation based on Delta table implementation seems to expect the table gets implicitly created if it is path based. I tried to fix it with the following change in SparkCatalog, which works with both read and write. But there is a catch because the initial table was created with an empty schema. So the second write always need to enable option("merge-schema", "true"). Any thoughts or suggestions to fix this issue? ` @@ -168,7 +176,19 @@ public class SparkCatalog extends BaseCatalog { try { return load(ident); } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { - throw new NoSuchTableException(ident); + if (isPathIdentifier(ident)) { + Schema schema = new Schema(Collections.emptyList()); + Catalog.TableBuilder builder = + tables.buildTable(((PathIdentifier) ident).location(), schema); + Map<String, String> properties = new HashMap<>(); + properties.put(TableProperties.SPARK_WRITE_ACCEPT_ANY_SCHEMA, "True"); + builder.withProperties(properties); + builder.withLocation(((PathIdentifier) ident).location()); + org.apache.iceberg.Table icebergTable = builder.create(); + return new SparkTable(icebergTable, !cacheEnabled); + } else { + throw new NoSuchTableException(ident); + } } } ` ### Willingness to contribute - [x] I can contribute a fix for this bug independently - [x] I would be willing to contribute a fix for this bug with guidance from the Iceberg community - [ ] I cannot contribute a fix for this bug at this time -- 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