diff --git a/ibis/backends/bigquery/__init__.py b/ibis/backends/bigquery/__init__.py index 9aefda52baa3..df1b2f3674d9 100644 --- a/ibis/backends/bigquery/__init__.py +++ b/ibis/backends/bigquery/__init__.py @@ -174,8 +174,8 @@ def _in_memory_table_exists(self, name: str) -> bool: table_ref = bq.TableReference(self._session_dataset, name) try: - self.client.get_table(table_ref) - except google.api_core.exceptions.NotFound: + self._get_table(table_ref) + except com.TableNotFound: return False else: return True @@ -619,12 +619,11 @@ def table( project, dataset = self._parse_project_and_dataset(database) - bq_table = self.client.get_table( - bq.TableReference( - bq.DatasetReference(project=project, dataset_id=dataset), - table.name, - ) + table_ref = bq.TableReference( + bq.DatasetReference(project=project, dataset_id=dataset), + table.name, ) + bq_table = self._get_table(table_ref) node = ops.DatabaseTable( table.name, @@ -636,6 +635,12 @@ def table( table_expr = node.to_expr() return rename_partitioned_column(table_expr, bq_table, self.partition_column) + def _get_table(self, table_ref: bq.TableReference): + try: + return self.client.get_table(table_ref) + except google.api_core.exceptions.NotFound as e: + raise com.TableNotFound(str(table_ref)) from e + def _make_session(self) -> tuple[str, str]: if (client := getattr(self, "client", None)) is not None: job_config = bq.QueryJobConfig(use_query_cache=False) @@ -867,8 +872,11 @@ def get_schema( ), name, ) + + table = self._get_table(table_ref) + return schema_from_bigquery_table( - self.client.get_table(table_ref), + table, # https://cloud.google.com/bigquery/docs/querying-wildcard-tables#filtering_selected_tables_using_table_suffix wildcard=name[-1] == "*", ) diff --git a/ibis/backends/clickhouse/__init__.py b/ibis/backends/clickhouse/__init__.py index 11f9bc85ac92..be3334373594 100644 --- a/ibis/backends/clickhouse/__init__.py +++ b/ibis/backends/clickhouse/__init__.py @@ -3,6 +3,7 @@ import ast import contextlib import glob +import re from contextlib import closing from functools import partial from typing import TYPE_CHECKING, Any, Literal @@ -14,6 +15,7 @@ import sqlglot as sg import sqlglot.expressions as sge import toolz +from clickhouse_connect.driver.exceptions import DatabaseError from clickhouse_connect.driver.external import ExternalData import ibis @@ -510,8 +512,13 @@ def get_schema( "`catalog` namespaces are not supported by clickhouse" ) query = sge.Describe(this=sg.table(table_name, db=database)) - with self._safe_raw_sql(query) as results: - names, types, *_ = results.result_columns + try: + with self._safe_raw_sql(query) as results: + names, types, *_ = results.result_columns + except DatabaseError as e: + if re.search(r"\bUNKNOWN_TABLE\b", str(e)): + raise com.TableNotFound(table_name) from e + return sch.Schema( dict(zip(names, map(self.compiler.type_mapper.from_string, types))) ) diff --git a/ibis/backends/datafusion/__init__.py b/ibis/backends/datafusion/__init__.py index ae00eecbdc94..8572ceadfe22 100644 --- a/ibis/backends/datafusion/__init__.py +++ b/ibis/backends/datafusion/__init__.py @@ -340,6 +340,9 @@ def get_schema( else: database = catalog.database() + if table_name not in database.names(): + raise com.TableNotFound(table_name) + table = database.table(table_name) return sch.schema(table.schema) diff --git a/ibis/backends/druid/__init__.py b/ibis/backends/druid/__init__.py index 3c593a77fa29..1fe40a38279b 100644 --- a/ibis/backends/druid/__init__.py +++ b/ibis/backends/druid/__init__.py @@ -9,14 +9,17 @@ import pydruid.db import sqlglot as sg +import sqlglot.expressions as sge import ibis.backends.sql.compilers as sc +import ibis.common.exceptions as com import ibis.expr.datatypes as dt import ibis.expr.schema as sch from ibis import util from ibis.backends.sql import SQLBackend from ibis.backends.sql.compilers.base import STAR from ibis.backends.sql.datatypes import DruidType +from ibis.backends.tests.errors import PyDruidProgrammingError if TYPE_CHECKING: from collections.abc import Iterable, Mapping @@ -147,6 +150,21 @@ def _get_schema_using_query(self, query: str) -> sch.Schema: schema[name] = dtype return sch.Schema(schema) + def _table_exists(self, name: str): + quoted = self.compiler.quoted + t = sg.table("TABLES", db="INFORMATION_SCHEMA", quoted=quoted) + table_name = sg.column("TABLE_NAME", quoted=quoted) + query = ( + sg.select(table_name) + .from_(t) + .where(table_name.eq(sge.convert(name))) + .sql(self.dialect) + ) + + with self._safe_raw_sql(query) as result: + tables = result.fetchall() + return bool(tables) + def get_schema( self, table_name: str, @@ -154,11 +172,19 @@ def get_schema( catalog: str | None = None, database: str | None = None, ) -> sch.Schema: - return self._get_schema_using_query( + query = ( sg.select(STAR) .from_(sg.table(table_name, db=database, catalog=catalog)) .sql(self.dialect) ) + try: + schema = self._get_schema_using_query(query) + except PyDruidProgrammingError as e: + if not self._table_exists(table_name): + raise com.TableNotFound(table_name) from e + raise + + return schema def _fetch_from_cursor(self, cursor, schema: sch.Schema) -> pd.DataFrame: import pandas as pd diff --git a/ibis/backends/duckdb/__init__.py b/ibis/backends/duckdb/__init__.py index 74f7e44ceff4..bb4d4f7aa9fd 100644 --- a/ibis/backends/duckdb/__init__.py +++ b/ibis/backends/duckdb/__init__.py @@ -302,7 +302,7 @@ def get_schema( try: result = self.con.sql(query) except duckdb.CatalogException: - raise exc.IbisError(f"Table not found: {table_name!r}") + raise exc.TableNotFound(table_name) else: meta = result.fetch_arrow_table() diff --git a/ibis/backends/exasol/__init__.py b/ibis/backends/exasol/__init__.py index 8218ac23fe31..ee381b2b363c 100644 --- a/ibis/backends/exasol/__init__.py +++ b/ibis/backends/exasol/__init__.py @@ -224,7 +224,7 @@ def get_schema( catalog: str | None = None, database: str | None = None, ) -> sch.Schema: - return self._get_schema_using_query( + query = ( sg.select(STAR) .from_( sg.table( @@ -236,6 +236,12 @@ def get_schema( ) .sql(self.dialect) ) + try: + return self._get_schema_using_query(query) + except pyexasol.exceptions.ExaQueryError as e: + if not self.con.meta.table_exists(table_name): + raise com.TableNotFound(table_name) from e + raise def _fetch_from_cursor(self, cursor, schema: sch.Schema) -> pd.DataFrame: import pandas as pd diff --git a/ibis/backends/flink/__init__.py b/ibis/backends/flink/__init__.py index 5772a83eeb1f..97ea8a06a8f5 100644 --- a/ibis/backends/flink/__init__.py +++ b/ibis/backends/flink/__init__.py @@ -1,6 +1,7 @@ from __future__ import annotations import itertools +import re from typing import TYPE_CHECKING, Any import sqlglot as sg @@ -302,7 +303,21 @@ def get_schema( qualified_name = sg.table(table_name, db=catalog, catalog=database).sql( self.name ) - table = self._table_env.from_path(qualified_name) + try: + table = self._table_env.from_path(qualified_name) + except Py4JJavaError as e: + # This seems too msg specific but not sure what a good work around is + # + # Flink doesn't have a way to check whether a table exists other + # than to all tables and check potentially every element in the list + if re.search( + "table .+ was not found", + str(e.java_exception.toString()), + flags=re.IGNORECASE, + ): + raise exc.TableNotFound(table_name) from e + raise + pyflink_schema = table.get_schema() return sch.Schema.from_pyarrow( diff --git a/ibis/backends/impala/__init__.py b/ibis/backends/impala/__init__.py index 4fe679db70df..522c0538bf17 100644 --- a/ibis/backends/impala/__init__.py +++ b/ibis/backends/impala/__init__.py @@ -383,14 +383,18 @@ def get_schema( Ibis schema """ - query = sge.Describe( - this=sg.table( - table_name, db=database, catalog=catalog, quoted=self.compiler.quoted - ) + table = sg.table( + table_name, db=database, catalog=catalog, quoted=self.compiler.quoted ) + with contextlib.closing(self.con.cursor()) as cur: + if not cur.table_exists(table_name, database_name=database or catalog): + raise com.TableNotFound(table.sql(self.dialect)) + + query = sge.Describe(this=table) with self._safe_raw_sql(query) as cur: meta = fetchall(cur) + return sch.Schema.from_tuples( zip(meta["name"], meta["type"].map(self.compiler.type_mapper.from_string)) ) diff --git a/ibis/backends/mssql/__init__.py b/ibis/backends/mssql/__init__.py index 3a1f57714299..3a215e1ece0b 100644 --- a/ibis/backends/mssql/__init__.py +++ b/ibis/backends/mssql/__init__.py @@ -278,7 +278,7 @@ def get_schema( if not meta: fqn = sg.table(name, db=database, catalog=catalog).sql(self.dialect) - raise com.IbisError(f"Table not found: {fqn}") + raise com.TableNotFound(fqn) mapping = {} for ( diff --git a/ibis/backends/mysql/__init__.py b/ibis/backends/mysql/__init__.py index 84c28432b151..664a1eff1da8 100644 --- a/ibis/backends/mysql/__init__.py +++ b/ibis/backends/mysql/__init__.py @@ -14,6 +14,7 @@ import sqlglot as sg import sqlglot.expressions as sge from pymysql.constants import ER +from pymysql.err import ProgrammingError import ibis import ibis.backends.sql.compilers as sc @@ -208,7 +209,6 @@ def _get_schema_using_query(self, query: str) -> sch.Schema: .limit(0) .sql(self.dialect) ) - return sch.Schema( { field.name: _type_from_cursor_info(descr, field) @@ -224,8 +224,13 @@ def get_schema( ).sql(self.dialect) with self.begin() as cur: - cur.execute(sge.Describe(this=table).sql(self.dialect)) - result = cur.fetchall() + try: + cur.execute(sge.Describe(this=table).sql(self.dialect)) + except ProgrammingError as e: + if e.args[0] == ER.NO_SUCH_TABLE: + raise com.TableNotFound(name) from e + else: + result = cur.fetchall() type_mapper = self.compiler.type_mapper fields = { diff --git a/ibis/backends/oracle/__init__.py b/ibis/backends/oracle/__init__.py index ed08714597de..6bcfe69b9591 100644 --- a/ibis/backends/oracle/__init__.py +++ b/ibis/backends/oracle/__init__.py @@ -382,7 +382,7 @@ def get_schema( results = cur.fetchall() if not results: - raise exc.IbisError(f"Table not found: {name!r}") + raise exc.TableNotFound(name) type_mapper = self.compiler.type_mapper fields = { diff --git a/ibis/backends/polars/__init__.py b/ibis/backends/polars/__init__.py index e0461ac8202b..9e9c25ceb40a 100644 --- a/ibis/backends/polars/__init__.py +++ b/ibis/backends/polars/__init__.py @@ -87,7 +87,11 @@ def list_tables(self, like=None, database=None): return self._filter_with_like(list(self._tables.keys()), like) def table(self, name: str) -> ir.Table: - schema = sch.infer(self._tables[name]) + table = self._tables.get(name) + if table is None: + raise com.TableNotFound(name) + + schema = sch.infer(table) return ops.DatabaseTable(name, schema, self).to_expr() def _in_memory_table_exists(self, name: str) -> bool: diff --git a/ibis/backends/postgres/__init__.py b/ibis/backends/postgres/__init__.py index ad7f51508e75..68feacd6c02c 100644 --- a/ibis/backends/postgres/__init__.py +++ b/ibis/backends/postgres/__init__.py @@ -546,7 +546,7 @@ def get_schema( rows = cur.fetchall() if not rows: - raise com.IbisError(f"Table not found: {name!r}") + raise com.TableNotFound(name) return sch.Schema( { diff --git a/ibis/backends/postgres/tests/test_client.py b/ibis/backends/postgres/tests/test_client.py index 90b018f102c1..c996c33f33ac 100644 --- a/ibis/backends/postgres/tests/test_client.py +++ b/ibis/backends/postgres/tests/test_client.py @@ -137,7 +137,7 @@ def test_create_and_drop_table(con, temp_table, params): con.drop_table(temp_table, **params) - with pytest.raises(com.IbisError): + with pytest.raises(com.TableNotFound, match=temp_table): con.table(temp_table, **params) diff --git a/ibis/backends/pyspark/__init__.py b/ibis/backends/pyspark/__init__.py index c19958e3db38..063fbc214d0a 100644 --- a/ibis/backends/pyspark/__init__.py +++ b/ibis/backends/pyspark/__init__.py @@ -11,6 +11,13 @@ import sqlglot.expressions as sge from packaging.version import parse as vparse from pyspark import SparkConf + +try: + from pyspark.errors.exceptions.base import AnalysisException # PySpark 3.5+ +except ImportError: + from pyspark.sql.utils import AnalysisException # PySpark 3.3 + + from pyspark.sql import SparkSession from pyspark.sql.types import BooleanType, DoubleType, LongType, StringType @@ -542,7 +549,13 @@ def get_schema( table_loc = self._to_sqlglot_table((catalog, database)) catalog, db = self._to_catalog_db_tuple(table_loc) with self._active_catalog_database(catalog, db): - df = self._session.table(table_name) + try: + df = self._session.table(table_name) + except AnalysisException as e: + if not self._session.catalog.tableExists(table_name): + raise com.TableNotFound(table_name) from e + raise + struct = PySparkType.to_ibis(df.schema) return sch.Schema(struct) diff --git a/ibis/backends/snowflake/__init__.py b/ibis/backends/snowflake/__init__.py index 93522f9b2113..0d62bb76f438 100644 --- a/ibis/backends/snowflake/__init__.py +++ b/ibis/backends/snowflake/__init__.py @@ -538,6 +538,8 @@ def get_schema( catalog: str | None = None, database: str | None = None, ) -> Iterable[tuple[str, dt.DataType]]: + import snowflake.connector + # this will always show temp tables with the same name as a non-temp # table first # @@ -548,8 +550,25 @@ def get_schema( table = sg.table( table_name, db=database, catalog=catalog, quoted=self.compiler.quoted ) - with self._safe_raw_sql(sge.Describe(kind="TABLE", this=table)) as cur: - result = cur.fetchall() + query = sge.Describe(kind="TABLE", this=table) + + try: + with self._safe_raw_sql(query) as cur: + result = cur.fetchall() + except snowflake.connector.errors.ProgrammingError as e: + # apparently sqlstate codes are "standard", in the same way that + # SQL is standard, because sqlstate codes are part of the SQL + # standard + # + # Nowhere does this exist in Snowflake's documentation but this + # exists in MariaDB's docs and matches the SQLSTATE error code + # + # https://mariadb.com/kb/en/sqlstate/ + # https://mariadb.com/kb/en/mariadb-error-code-reference/ + # and the least helpful version: https://docs.snowflake.com/en/developer-guide/snowflake-scripting/exceptions#handling-an-exception + if e.sqlstate == "42S02": + raise com.TableNotFound(table.sql(self.dialect)) from e + raise type_mapper = self.compiler.type_mapper return sch.Schema( diff --git a/ibis/backends/sqlite/__init__.py b/ibis/backends/sqlite/__init__.py index cb5d252b054f..a1f4f078178d 100644 --- a/ibis/backends/sqlite/__init__.py +++ b/ibis/backends/sqlite/__init__.py @@ -233,7 +233,7 @@ def _inspect_schema( cur.execute(sql) rows = cur.fetchall() if not rows: - raise com.IbisError(f"Table not found: {table_name!r}") + raise com.TableNotFound(table_name) table_info = {name: (typ, not notnull) for name, typ, notnull in rows} diff --git a/ibis/backends/tests/test_array.py b/ibis/backends/tests/test_array.py index 190d89a6c803..3b440feb2293 100644 --- a/ibis/backends/tests/test_array.py +++ b/ibis/backends/tests/test_array.py @@ -19,7 +19,6 @@ from ibis.backends.tests.errors import ( ClickHouseDatabaseError, GoogleBadRequest, - ImpalaHiveServer2Error, MySQLOperationalError, PolarsComputeError, PsycoPg2IndeterminateDatatype, @@ -64,7 +63,7 @@ raises=( com.UnsupportedBackendType, com.OperationNotDefinedError, - ImpalaHiveServer2Error, + com.TableNotFound, ), ), pytest.mark.notimpl(["druid", "oracle"], raises=Exception), diff --git a/ibis/backends/tests/test_client.py b/ibis/backends/tests/test_client.py index 2d18a65fcf09..8e3618b4454b 100644 --- a/ibis/backends/tests/test_client.py +++ b/ibis/backends/tests/test_client.py @@ -26,17 +26,13 @@ import ibis.expr.operations as ops from ibis.backends.conftest import ALL_BACKENDS from ibis.backends.tests.errors import ( - ClickHouseDatabaseError, ExaQueryError, - GoogleNotFound, ImpalaHiveServer2Error, - MySQLProgrammingError, OracleDatabaseError, PsycoPg2InternalError, PsycoPg2UndefinedObject, Py4JJavaError, PyODBCProgrammingError, - PySparkAnalysisException, SnowflakeProgrammingError, ) from ibis.util import gen_name @@ -1602,6 +1598,11 @@ def test_from_connection(con, top_level): assert result == 1 +def test_table_not_found(con): + with pytest.raises(com.TableNotFound): + con.table(gen_name("table_not_found")) + + @pytest.mark.notimpl( ["flink"], raises=com.IbisError, reason="not yet implemented for Flink" ) @@ -1624,33 +1625,8 @@ def test_no_accidental_cross_database_table_load(con_create_database): con.drop_table(table) - # NOTE: this entire block of exception type munging goes away once we unify - # table-not-found exceptions - - # always allowed to raise - always_allowed = (com.IbisError,) - - # these exception types are None when the backend dependency that - # defines them is not installed - allowed_when_installed = filter( - None, - ( - ClickHouseDatabaseError, - PySparkAnalysisException, - MySQLProgrammingError, - ExaQueryError, - SnowflakeProgrammingError, - GoogleNotFound, - ), - ) - - # we only want to allow base Exception when we're testing datafusion - # otherwise any exceptions, including those that are unrelated to the - # problem under test will be considered correctly raising - datafusion_only = (Exception,) * (con.name == "datafusion") - # Now attempting to load same table name without specifying db should fail - with pytest.raises((*always_allowed, *allowed_when_installed, *datafusion_only)): + with pytest.raises(com.TableNotFound): t = con.table(table) # But can load if specify other db diff --git a/ibis/backends/tests/test_join.py b/ibis/backends/tests/test_join.py index bb7b1c9915d4..88fdd5489384 100644 --- a/ibis/backends/tests/test_join.py +++ b/ibis/backends/tests/test_join.py @@ -9,7 +9,6 @@ import ibis import ibis.common.exceptions as com import ibis.expr.schema as sch -from ibis.backends.tests.errors import PyDruidProgrammingError np = pytest.importorskip("numpy") pd = pytest.importorskip("pandas") @@ -271,7 +270,7 @@ def test_join_with_trivial_predicate(awards_players, predicate, how, pandas_valu assert len(result) == len(expected) -@pytest.mark.notimpl(["druid"], raises=PyDruidProgrammingError) +@pytest.mark.notimpl(["druid"], raises=com.TableNotFound) @pytest.mark.parametrize( ("how", "nrows", "gen_right", "keys"), [ diff --git a/ibis/backends/tests/test_window.py b/ibis/backends/tests/test_window.py index 083254bf2e25..0942160ad1c0 100644 --- a/ibis/backends/tests/test_window.py +++ b/ibis/backends/tests/test_window.py @@ -27,7 +27,12 @@ pytestmark = [ pytest.mark.notimpl( - ["druid"], raises=(com.OperationNotDefinedError, PyDruidProgrammingError) + ["druid"], + raises=( + com.OperationNotDefinedError, + com.TableNotFound, + PyDruidProgrammingError, + ), ) ] diff --git a/ibis/backends/trino/__init__.py b/ibis/backends/trino/__init__.py index 444a23fa90bc..61c3c9a4b223 100644 --- a/ibis/backends/trino/__init__.py +++ b/ibis/backends/trino/__init__.py @@ -159,7 +159,7 @@ def get_schema( if not meta: fqn = sg.table(table_name, db=database, catalog=catalog).sql(self.name) - raise com.IbisError(f"Table not found: {fqn}") + raise com.TableNotFound(fqn) type_mapper = self.compiler.type_mapper diff --git a/ibis/backends/trino/tests/test_client.py b/ibis/backends/trino/tests/test_client.py index 6c4c00a51cd7..33a019c463e9 100644 --- a/ibis/backends/trino/tests/test_client.py +++ b/ibis/backends/trino/tests/test_client.py @@ -164,7 +164,7 @@ def test_table_access_database_schema(con): t = con.table("region", database=("tpch", "sf1")) assert t.count().execute() - with pytest.raises(exc.IbisError, match='Table not found: tpch."tpch.sf1".region'): + with pytest.raises(exc.TableNotFound, match=r".*region"): con.table("region", database=("tpch", "tpch.sf1")) with pytest.raises(exc.IbisError, match="Overspecified table hierarchy provided"): diff --git a/ibis/common/exceptions.py b/ibis/common/exceptions.py index 0c03d5cd4a9e..745614b32484 100644 --- a/ibis/common/exceptions.py +++ b/ibis/common/exceptions.py @@ -21,6 +21,10 @@ from collections.abc import Callable +class TableNotFound(Exception): + """Exception to raise when a table cannot be found.""" + + class IbisError(Exception): """IbisError."""