From c4da376348a62524c9db815ae16ffc6b6f173547 Mon Sep 17 00:00:00 2001 From: Jacky Lee Date: Mon, 23 Mar 2026 17:43:16 +0800 Subject: [PATCH 1/3] Docs: clarify SparkSessionCatalog function limitations --- docs/docs/spark-configuration.md | 7 +++++++ docs/docs/spark-queries.md | 8 ++++++++ 2 files changed, 15 insertions(+) diff --git a/docs/docs/spark-configuration.md b/docs/docs/spark-configuration.md index 01bb773680fd..d5f725a8ebc6 100644 --- a/docs/docs/spark-configuration.md +++ b/docs/docs/spark-configuration.md @@ -112,6 +112,13 @@ Spark's built-in catalog supports existing v1 and v2 tables tracked in a Hive Me This configuration can use same Hive Metastore for both Iceberg and non-Iceberg tables. +`SparkSessionCatalog` is useful when you want `spark_catalog` to work with both Iceberg and non-Iceberg +tables in the same metastore. It is not a full replacement for a dedicated Iceberg catalog, though. +In Spark versions before 4.2.0, `SparkSessionCatalog` does not expose Iceberg's `system` namespace +(see SPARK-54760), so catalog-scoped SQL functions such as `system.bucket`, `system.days`, and +`system.iceberg_version` are not available through `spark_catalog`. To use those functions, configure a +separate Iceberg catalog with `org.apache.iceberg.spark.SparkCatalog` and call them through that catalog. + ### Using catalog specific Hadoop configuration values Similar to configuring Hadoop properties by using `spark.hadoop.*`, it's possible to set per-catalog Hadoop configuration values when using Spark by adding the property for the catalog with the prefix `spark.sql.catalog.(catalog-name).hadoop.*`. These properties will take precedence over values configured globally using `spark.hadoop.*` and will only affect Iceberg tables. diff --git a/docs/docs/spark-queries.md b/docs/docs/spark-queries.md index 54c3b1572dac..d39036c159e1 100644 --- a/docs/docs/spark-queries.md +++ b/docs/docs/spark-queries.md @@ -51,6 +51,14 @@ writing filters that match Iceberg partition transforms. These functions are ava [Iceberg catalog](spark-configuration.md#catalog-configuration); they are not registered in Spark's built-in catalog. +!!! note + In Spark versions before 4.2.0, `SparkSessionCatalog` does not expose Iceberg's `system` + namespace (see SPARK-54760). Queries such as `SELECT spark_catalog.system.bucket(16, id)` + fail even when `spark_catalog` is configured with + `org.apache.iceberg.spark.SparkSessionCatalog`. + To use Iceberg SQL functions, call them through a catalog configured with + `org.apache.iceberg.spark.SparkCatalog`. + Use the `system` namespace when calling these functions: ```sql From b8d41a7d2afc5dd6f9ea702b26b07a5591b4fcc5 Mon Sep 17 00:00:00 2001 From: Jacky Lee Date: Tue, 24 Mar 2026 17:12:43 +0800 Subject: [PATCH 2/3] Docs: explain SparkSessionCatalog V2Function limitation --- docs/docs/spark-configuration.md | 9 +++++---- docs/docs/spark-queries.md | 8 ++++---- 2 files changed, 9 insertions(+), 8 deletions(-) diff --git a/docs/docs/spark-configuration.md b/docs/docs/spark-configuration.md index d5f725a8ebc6..7b26df9a2685 100644 --- a/docs/docs/spark-configuration.md +++ b/docs/docs/spark-configuration.md @@ -114,10 +114,11 @@ This configuration can use same Hive Metastore for both Iceberg and non-Iceberg `SparkSessionCatalog` is useful when you want `spark_catalog` to work with both Iceberg and non-Iceberg tables in the same metastore. It is not a full replacement for a dedicated Iceberg catalog, though. -In Spark versions before 4.2.0, `SparkSessionCatalog` does not expose Iceberg's `system` namespace -(see SPARK-54760), so catalog-scoped SQL functions such as `system.bucket`, `system.days`, and -`system.iceberg_version` are not available through `spark_catalog`. To use those functions, configure a -separate Iceberg catalog with `org.apache.iceberg.spark.SparkCatalog` and call them through that catalog. +Spark before 4.2.0 does not support `V2Function` in the session catalog. See +[SPARK-54760](https://issues.apache.org/jira/browse/SPARK-54760) for details. As a result, +catalog-scoped SQL functions such as `system.bucket`, `system.days`, and `system.iceberg_version` +are not available through `spark_catalog`. To workaround this limitation, configure a separate +Iceberg catalog with `org.apache.iceberg.spark.SparkCatalog` and call them through that catalog. ### Using catalog specific Hadoop configuration values diff --git a/docs/docs/spark-queries.md b/docs/docs/spark-queries.md index d39036c159e1..ca7face3ce03 100644 --- a/docs/docs/spark-queries.md +++ b/docs/docs/spark-queries.md @@ -52,10 +52,10 @@ writing filters that match Iceberg partition transforms. These functions are ava built-in catalog. !!! note - In Spark versions before 4.2.0, `SparkSessionCatalog` does not expose Iceberg's `system` - namespace (see SPARK-54760). Queries such as `SELECT spark_catalog.system.bucket(16, id)` - fail even when `spark_catalog` is configured with - `org.apache.iceberg.spark.SparkSessionCatalog`. + Spark before 4.2.0 does not support `V2Function` in the session catalog. + Queries such as `SELECT spark_catalog.system.bucket(16, id)` fail even when + `spark_catalog` is configured with `org.apache.iceberg.spark.SparkSessionCatalog`. + See [SPARK-54760](https://issues.apache.org/jira/browse/SPARK-54760) for details. To use Iceberg SQL functions, call them through a catalog configured with `org.apache.iceberg.spark.SparkCatalog`. From ccb9dae17be48853e96ee95ea7be560d6d146877 Mon Sep 17 00:00:00 2001 From: Jacky Lee Date: Thu, 26 Mar 2026 17:58:56 +0800 Subject: [PATCH 3/3] Docs: fix comments --- docs/docs/spark-configuration.md | 14 ++++++++------ docs/docs/spark-queries.md | 2 +- 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/docs/docs/spark-configuration.md b/docs/docs/spark-configuration.md index 7b26df9a2685..e8e4f7e3c8c1 100644 --- a/docs/docs/spark-configuration.md +++ b/docs/docs/spark-configuration.md @@ -113,12 +113,14 @@ Spark's built-in catalog supports existing v1 and v2 tables tracked in a Hive Me This configuration can use same Hive Metastore for both Iceberg and non-Iceberg tables. `SparkSessionCatalog` is useful when you want `spark_catalog` to work with both Iceberg and non-Iceberg -tables in the same metastore. It is not a full replacement for a dedicated Iceberg catalog, though. -Spark before 4.2.0 does not support `V2Function` in the session catalog. See -[SPARK-54760](https://issues.apache.org/jira/browse/SPARK-54760) for details. As a result, -catalog-scoped SQL functions such as `system.bucket`, `system.days`, and `system.iceberg_version` -are not available through `spark_catalog`. To workaround this limitation, configure a separate -Iceberg catalog with `org.apache.iceberg.spark.SparkCatalog` and call them through that catalog. +tables in the same metastore. + +!!! note + Spark before 4.2.0 does not support `V2Function` in the session catalog. See + [SPARK-54760](https://issues.apache.org/jira/browse/SPARK-54760) ([apache/spark#53531](https://github.com/apache/spark/pull/53531)) for details. As a result, + catalog-scoped SQL functions such as `system.bucket`, `system.days`, and `system.iceberg_version` + are not available through `spark_catalog`. To work around this limitation, configure a separate + Iceberg catalog with `org.apache.iceberg.spark.SparkCatalog` and call them through that catalog. ### Using catalog specific Hadoop configuration values diff --git a/docs/docs/spark-queries.md b/docs/docs/spark-queries.md index ca7face3ce03..91f1759f568c 100644 --- a/docs/docs/spark-queries.md +++ b/docs/docs/spark-queries.md @@ -55,7 +55,7 @@ built-in catalog. Spark before 4.2.0 does not support `V2Function` in the session catalog. Queries such as `SELECT spark_catalog.system.bucket(16, id)` fail even when `spark_catalog` is configured with `org.apache.iceberg.spark.SparkSessionCatalog`. - See [SPARK-54760](https://issues.apache.org/jira/browse/SPARK-54760) for details. + See [SPARK-54760](https://issues.apache.org/jira/browse/SPARK-54760) ([apache/spark#53531](https://github.com/apache/spark/pull/53531)) for details. To use Iceberg SQL functions, call them through a catalog configured with `org.apache.iceberg.spark.SparkCatalog`.