Skip to content

Conversation

@stuxuhai
Copy link
Contributor

This is a follow-up PR. The previous PR was closed after the branch was force-reset to apache:main.

Purpose

This PR fixes a bug where CREATE VIEW IF NOT EXISTS fails with a NoSuchIcebergViewException: Not an iceberg view (wrapped in QueryExecutionException) instead of succeeding silently when a non-Iceberg view (e.g., a Hive view) already exists in the SparkSessionCatalog.

The Problem

When SparkSessionCatalog is configured with spark.sql.extensions=org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions
spark.sql.catalog.spark_catalog=org.apache.iceberg.spark.SparkSessionCatalog
spark.sql.catalog.spark_catalog.type=hive

  1. A user executes CREATE VIEW IF NOT EXISTS db.view_name AS ....
  2. If db.view_name already exists as a Hive View (or any non-Iceberg table/view).
  3. SparkSessionCatalog.createView currently delegates directly to the underlying Iceberg catalog (asViewCatalog.createView).
  4. The Iceberg catalog (e.g., HiveCatalog) attempts to load the view. Since it is not an Iceberg view, it throws NoSuchIcebergViewException.
  5. Spark expects ViewAlreadyExistsException to handle the IF NOT EXISTS logic. Because it receives a different exception, the query fails entirely.

The Fix

Before delegating the creation to the Iceberg catalog, we explicitly check if the identifier already exists in the underlying session catalog (which is the source of truth for the global namespace).

If getSessionCatalog().tableExists(ident) returns true, we immediately throw ViewAlreadyExistsException. This allows Spark's analysis rules to correctly catch the exception and ignore the operation as per IF NOT EXISTS semantics.

Verification

  • Added a new unit test in TestSparkSessionCatalog to verify that CREATE VIEW IF NOT EXISTS succeeds when a Hive view exists.
  • Verified that CREATE VIEW (without if not exists) correctly throws AnalysisException (Table or view already exists).

@github-actions github-actions bot added the spark label Dec 27, 2025
@stuxuhai
Copy link
Contributor Author

stuxuhai commented Jan 6, 2026

@huaxingao The previous PR was automatically closed due to a force push, so I’ve opened a new one.
Could you please help review it when you have time? Thanks!

import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;

public class TestSparkSessionCatalogWithExtensions {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would suggest to first fix the issue in one Spark version and later backport stuff

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1 to first fix 4.1 and then back-porting

@nastra nastra requested a review from huaxingao January 8, 2026 16:04
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;

public class TestSparkSessionCatalogWithExtensions {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1 to first fix 4.1 and then back-porting

}
}

public static void setUpCatalog() {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: private?

spark.conf().set("spark.sql.catalog.spark_catalog.type", "hive");
}

public static void resetSparkCatalog() {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: private?

protected static TestHiveMetastore metastore = null;
protected static HiveConf hiveConf = null;
protected static SparkSession spark = null;
protected static JavaSparkContext sparkContext = null;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

is this necessary? If not, can we remove?

spark
.conf()
.set("spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkSessionCatalog");
spark.conf().set("spark.sql.catalog.spark_catalog.type", "hive");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we add spark.sessionState().catalogManager().reset() when flipping these configs (either inside the helper methods or immediately after calling them in the tests), similar to how spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestSparkSessionCatalog.java does it?

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants