Skip to content

Commit

Permalink
test: consolidate backend errors in a single file
Browse files Browse the repository at this point in the history
  • Loading branch information
cpcloud committed Dec 29, 2023
1 parent 83fe326 commit e88388f
Show file tree
Hide file tree
Showing 17 changed files with 169 additions and 349 deletions.
6 changes: 1 addition & 5 deletions ibis/backends/flink/tests/test_ddl.py
Original file line number Diff line number Diff line change
Expand Up @@ -11,11 +11,7 @@
import ibis.expr.datatypes as dt
import ibis.expr.schema as sch
from ibis.backends.conftest import TEST_TABLES

try:
from py4j.protocol import Py4JJavaError
except ImportError:
Py4JJavaError = None
from ibis.backends.tests.errors import Py4JJavaError

_awards_players_schema = sch.Schema(
{
Expand Down
66 changes: 66 additions & 0 deletions ibis/backends/tests/errors.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,66 @@
from __future__ import annotations

try:
from duckdb import ConversionException as DuckDBConversionException
from duckdb import InvalidInputException as DuckDBInvalidInputException
except ImportError:
DuckDBConversionException = DuckDBInvalidInputException = None

try:
from clickhouse_connect.driver.exceptions import (
DatabaseError as ClickHouseDatabaseError,
)
from clickhouse_connect.driver.exceptions import (
InternalError as ClickHouseInternalError,
)
except ImportError:
ClickHouseDatabaseError = ClickHouseInternalError = None

try:
from pyexasol.exceptions import ExaQueryError
except ImportError:
ExaQueryError = None

try:
from pyspark.sql.utils import AnalysisException as PySparkAnalysisException
from pyspark.sql.utils import (
IllegalArgumentException as PySparkIllegalArgumentException,
)
from pyspark.sql.utils import PythonException as PySparkPythonException
except ImportError:
PySparkAnalysisException = (
PySparkIllegalArgumentException
) = PySparkPythonException = None

try:
from google.api_core.exceptions import BadRequest as GoogleBadRequest
except ImportError:
GoogleBadRequest = None

try:
from polars import ComputeError as PolarsComputeError
from polars import PanicException as PolarsPanicException
from polars.exceptions import InvalidOperationError as PolarsInvalidOperationError
except ImportError:
PolarsComputeError = PolarsPanicException = PolarsInvalidOperationError = None

try:
from pyarrow import ArrowInvalid, ArrowNotImplementedError
except ImportError:
ArrowInvalid = ArrowNotImplementedError = None

try:
from impala.error import HiveServer2Error as ImpalaHiveServer2Error
from impala.error import OperationalError as ImpalaOperationalError
except ImportError:
ImpalaHiveServer2Error = ImpalaOperationalError = None

try:
from py4j.protocol import Py4JError, Py4JJavaError
except ImportError:
Py4JJavaError = Py4JError = None

try:
from deltalake import PyDeltaTableError
except ImportError:
PyDeltaTableError = None
48 changes: 12 additions & 36 deletions ibis/backends/tests/test_aggregation.py
Original file line number Diff line number Diff line change
Expand Up @@ -14,40 +14,16 @@
import ibis.expr.datatypes as dt
from ibis import _
from ibis import literal as L
from ibis.backends.tests.errors import (
ClickHouseDatabaseError,
ExaQueryError,
GoogleBadRequest,
PolarsInvalidOperationError,
Py4JError,
PySparkAnalysisException,
)
from ibis.legacy.udf.vectorized import reduction

try:
from pyspark.sql.utils import AnalysisException as PysparkAnalysisException
except ImportError:
PysparkAnalysisException = None

try:
from google.api_core.exceptions import BadRequest as GoogleBadRequest
except ImportError:
GoogleBadRequest = None

try:
from clickhouse_connect.driver.exceptions import (
DatabaseError as ClickhouseDatabaseError,
)
except ImportError:
ClickhouseDatabaseError = None

try:
from py4j.protocol import Py4JError
except ImportError:
Py4JError = None

try:
from pyexasol.exceptions import ExaQueryError
except ImportError:
ExaQueryError = None

try:
from polars.exceptions import InvalidOperationError as PolarsInvalidOperationError
except ImportError:
PolarsInvalidOperationError = None


@reduction(input_type=[dt.double], output_type=dt.double)
def mean_udf(s):
Expand Down Expand Up @@ -410,7 +386,7 @@ def mean_and_std(v):
marks=[
pytest.mark.notimpl(
["pyspark"],
raises=PysparkAnalysisException,
raises=PySparkAnalysisException,
reason=(
"pyspark.sql.utils.AnalysisException: "
"function sum requires numeric or interval types, not boolean;"
Expand Down Expand Up @@ -1220,7 +1196,7 @@ def test_median(alltypes, df):
)
@pytest.mark.notyet(
["clickhouse"],
raises=ClickhouseDatabaseError,
raises=ClickHouseDatabaseError,
reason="doesn't support median of strings",
)
@pytest.mark.notyet(
Expand Down Expand Up @@ -1646,11 +1622,11 @@ def test_grouped_case(backend, con):
raises=AssertionError,
)
@pytest.mark.notyet(["impala", "flink"], raises=com.UnsupportedOperationError)
@pytest.mark.notyet(["clickhouse"], raises=ClickhouseDatabaseError)
@pytest.mark.notyet(["clickhouse"], raises=ClickHouseDatabaseError)
@pytest.mark.notyet(["druid", "trino", "snowflake"], raises=sa.exc.ProgrammingError)
@pytest.mark.notyet(["mysql"], raises=sa.exc.NotSupportedError)
@pytest.mark.notyet(["oracle"], raises=sa.exc.DatabaseError)
@pytest.mark.notyet(["pyspark"], raises=PysparkAnalysisException)
@pytest.mark.notyet(["pyspark"], raises=PySparkAnalysisException)
def test_group_concat_over_window(backend, con):
input_df = pd.DataFrame(
{
Expand Down
64 changes: 21 additions & 43 deletions ibis/backends/tests/test_array.py
Original file line number Diff line number Diff line change
Expand Up @@ -17,35 +17,13 @@
import ibis.common.exceptions as com
import ibis.expr.datatypes as dt
import ibis.expr.types as ir

try:
from clickhouse_connect.driver.exceptions import (
DatabaseError as ClickhouseDatabaseError,
)
except ImportError:
ClickhouseDatabaseError = None


try:
from google.api_core.exceptions import BadRequest
except ImportError:
BadRequest = None

try:
from pyspark.sql.utils import AnalysisException as PySparkAnalysisException
except ImportError:
PySparkAnalysisException = None

try:
from polars.exceptions import ComputeError as PolarsComputeError
except ImportError:
PolarsComputeError = None

try:
from py4j.protocol import Py4JJavaError
except ImportError:
Py4JJavaError = None

from ibis.backends.tests.errors import (
ClickHouseDatabaseError,
GoogleBadRequest,
PolarsComputeError,
Py4JJavaError,
PySparkAnalysisException,
)

pytestmark = [
pytest.mark.never(
Expand Down Expand Up @@ -239,7 +217,7 @@ def test_array_discovery(backend):
@pytest.mark.notyet(
["bigquery"],
reason="BigQuery doesn't support casting array<T> to array<U>",
raises=BadRequest,
raises=GoogleBadRequest,
)
@pytest.mark.notimpl(["dask"], raises=ValueError)
@pytest.mark.notimpl(["datafusion"], raises=com.OperationNotDefinedError)
Expand Down Expand Up @@ -415,7 +393,7 @@ def test_array_slice(backend, start, stop):
marks=[
pytest.mark.notyet(
["bigquery"],
raises=BadRequest,
raises=GoogleBadRequest,
reason="BigQuery doesn't support arrays with null elements",
)
],
Expand Down Expand Up @@ -460,7 +438,7 @@ def test_array_map(backend, con, input, output):
marks=[
pytest.mark.notyet(
["bigquery"],
raises=BadRequest,
raises=GoogleBadRequest,
reason="NULLs are not allowed as array elements",
)
],
Expand Down Expand Up @@ -532,7 +510,7 @@ def test_array_remove(backend, con):
)
@pytest.mark.notyet(
["bigquery"],
raises=BadRequest,
raises=GoogleBadRequest,
reason="BigQuery doesn't support arrays with null elements",
)
@pytest.mark.notyet(
Expand All @@ -542,7 +520,7 @@ def test_array_remove(backend, con):
)
@pytest.mark.notyet(
["bigquery"],
raises=(AssertionError, BadRequest),
raises=(AssertionError, GoogleBadRequest),
reason="bigquery doesn't support null elements in arrays",
)
@pytest.mark.parametrize(
Expand Down Expand Up @@ -588,7 +566,7 @@ def test_array_sort(backend, con):
)
@pytest.mark.notyet(
["bigquery"],
raises=BadRequest,
raises=GoogleBadRequest,
reason="BigQuery doesn't support arrays with null elements",
)
def test_array_union(con):
Expand Down Expand Up @@ -618,7 +596,7 @@ def test_array_union(con):
marks=[
pytest.mark.notyet(
["bigquery"],
raises=BadRequest,
raises=GoogleBadRequest,
reason="BigQuery doesn't support arrays with null elements",
)
],
Expand All @@ -643,7 +621,7 @@ def test_array_intersect(con, data):
@builtin_array
@pytest.mark.notimpl(
["clickhouse"],
raises=ClickhouseDatabaseError,
raises=ClickHouseDatabaseError,
reason="ClickHouse won't accept dicts for struct type values",
)
@pytest.mark.notimpl(["postgres"], raises=sa.exc.ProgrammingError)
Expand Down Expand Up @@ -688,7 +666,7 @@ def test_zip(backend):
@builtin_array
@pytest.mark.notyet(
["clickhouse"],
raises=ClickhouseDatabaseError,
raises=ClickHouseDatabaseError,
reason="https://github.com/ClickHouse/ClickHouse/issues/41112",
)
@pytest.mark.notimpl(["postgres"], raises=sa.exc.ProgrammingError)
Expand Down Expand Up @@ -765,7 +743,7 @@ def flatten_data():
pytest.mark.notyet(
["clickhouse"],
reason="doesn't support nullable array elements",
raises=ClickhouseDatabaseError,
raises=ClickHouseDatabaseError,
)
],
),
Expand All @@ -777,7 +755,7 @@ def flatten_data():
pytest.mark.notyet(
["clickhouse"],
reason="doesn't support nullable array elements",
raises=ClickhouseDatabaseError,
raises=ClickHouseDatabaseError,
)
],
),
Expand Down Expand Up @@ -857,7 +835,7 @@ def test_range_start_stop_step(con, start, stop, step):
@pytest.mark.parametrize("stop", [-7, 0, 7])
@pytest.mark.parametrize("start", [-7, 0, 7])
@pytest.mark.notyet(
["clickhouse"], raises=ClickhouseDatabaseError, reason="not supported upstream"
["clickhouse"], raises=ClickHouseDatabaseError, reason="not supported upstream"
)
@pytest.mark.notyet(
["datafusion"], raises=com.OperationNotDefinedError, reason="not supported upstream"
Expand Down Expand Up @@ -1010,7 +988,7 @@ def swap(token):
id="neg_outer",
marks=[
pytest.mark.notyet(["polars"], raises=com.UnsupportedOperationError),
pytest.mark.notyet(["bigquery"], raises=BadRequest),
pytest.mark.notyet(["bigquery"], raises=GoogleBadRequest),
pytest.mark.notyet(
["clickhouse", "pyspark", "snowflake"],
raises=com.UnsupportedOperationError,
Expand Down Expand Up @@ -1049,7 +1027,7 @@ def test_timestamp_range(con, start, stop, step, freq, tzinfo):
id="neg",
marks=[
pytest.mark.notyet(["polars"], raises=com.UnsupportedOperationError),
pytest.mark.notyet(["bigquery"], raises=BadRequest),
pytest.mark.notyet(["bigquery"], raises=GoogleBadRequest),
pytest.mark.notyet(
["clickhouse", "pyspark", "snowflake"],
raises=com.UnsupportedOperationError,
Expand Down
6 changes: 1 addition & 5 deletions ibis/backends/tests/test_client.py
Original file line number Diff line number Diff line change
Expand Up @@ -27,16 +27,12 @@
import ibis.expr.datatypes as dt
import ibis.expr.operations as ops
from ibis.backends.conftest import ALL_BACKENDS
from ibis.backends.tests.errors import Py4JJavaError
from ibis.util import gen_name, guid

if TYPE_CHECKING:
from ibis.backends.base import BaseBackend

try:
from py4j.protocol import Py4JJavaError
except ImportError:
Py4JJavaError = None


@pytest.fixture
def new_schema():
Expand Down
11 changes: 1 addition & 10 deletions ibis/backends/tests/test_dot_sql.py
Original file line number Diff line number Diff line change
Expand Up @@ -8,11 +8,7 @@
import ibis
from ibis import _
from ibis.backends.base import _IBIS_TO_SQLGLOT_DIALECT, _get_backend_names

try:
from polars.exceptions import ComputeError as PolarsComputeError
except ImportError:
PolarsComputeError = None
from ibis.backends.tests.errors import PolarsComputeError

table_dot_sql_notimpl = pytest.mark.notimpl(["bigquery", "impala", "druid"])
dot_sql_notimpl = pytest.mark.notimpl(["datafusion", "exasol", "flink"])
Expand All @@ -30,11 +26,6 @@
"bigquery": "ibis_gbq_testing.functional_alltypes",
}

try:
from clickhouse_connect.driver.exceptions import DatabaseError
except ImportError:
DatabaseError = None


@dot_sql_notimpl
@dot_sql_notyet
Expand Down
Loading

0 comments on commit e88388f

Please sign in to comment.