Skip to content

Commit

Permalink
Use cache to reduce redundant database calls (#1488)
Browse files Browse the repository at this point in the history
We were calling DB unnecessarily at various places to get the same info.
This can be cached, this is evident from the below logs where it is
reduced from 7 calls to 1 :

*Before 1*:

```
[2022-12-23 02:09:16,425] {dag.py:3622} INFO - Running task top_five_animations
[2022-12-23 02:09:16,438] {taskinstance.py:1511} INFO - Exporting the following env vars:
...
[2022-12-23 02:09:16,439] {base.py:73} INFO - Using connection ID 'sqlite_default' for task execution.
[2022-12-23 02:09:16,440] {base.py:73} INFO - Using connection ID 'sqlite_default' for task execution.
[2022-12-23 02:09:16,440] {base_decorator.py:124} INFO - Returning table Table(name='top_animation', conn_id='sqlite_default', metadata=Metadata(schema=None, database=None), columns=[], temp=False, uri='astro://@?table=top_animation', extra={})
[2022-12-23 02:09:16,440] {base_decorator.py:124} INFO - Returning table Table(name='top_animation', conn_id='sqlite_default', metadata=Metadata(schema=None, database=None), columns=[], temp=False, uri='astro://@?table=top_animation', extra={})
[2022-12-23 02:09:16,441] {base.py:73} INFO - Using connection ID 'sqlite_default' for task execution.
[2022-12-23 02:09:16,442] {base.py:73} INFO - Using connection ID 'sqlite_default' for task execution.
[2022-12-23 02:09:16,445] {base.py:73} INFO - Using connection ID 'sqlite_default' for task execution.
[2022-12-23 02:09:16,450] {base.py:73} INFO - Using connection ID 'sqlite_default' for task execution.
[2022-12-23 02:09:16,450] {base.py:73} INFO - Using connection ID 'sqlite_default' for task execution.
[2022-12-23 02:09:16,461] {taskinstance.py:1322} INFO - Marking task as SUCCESS. dag_id=calculate_popular_movies, task_id=top_five_animations, execution_date=20221223T020915, start_date=, end_date=20221223T020916
[2022-12-23 02:09:16,461] {taskinstance.py:1322} INFO - Marking task as SUCCESS. dag_id=calculate_popular_movies, task_id=top_five_animations, execution_date=20221223T020915, start_date=, end_date=20221223T020916
[2022-12-23 02:09:16,464] {dag.py:3626} INFO - top_five_animations ran successfully!
[2022-12-23 02:09:16,464] {dag.py:3629} INFO - *****************************************************
[2022-12-23 02:09:16,465] {dagrun.py:606} INFO - Marking run <DagRun calculate_popular_movies @ 2022-12-23T02:09:15.324979+00:00: manual__2022-12-23T02:09:15.324979+00:00, state:running, queued_at: None. externally triggered: False> successful
```

*After 1*:
```
[2022-12-23 02:20:18,669] {dag.py:3622} INFO - Running task top_five_animations
[2022-12-23 02:20:18,680] {taskinstance.py:1511} INFO - Exporting the following env vars:
...
[2022-12-23 02:20:18,681] {base_decorator.py:124} INFO - Returning table Table(name='top_animation', conn_id='sqlite_default', metadata=Metadata(schema=None, database=None), columns=[], temp=False, uri='astro://@?table=top_animation', extra={})
[2022-12-23 02:20:18,681] {base_decorator.py:124} INFO - Returning table Table(name='top_animation', conn_id='sqlite_default', metadata=Metadata(schema=None, database=None), columns=[], temp=False, uri='astro://@?table=top_animation', extra={})
[2022-12-23 02:20:18,686] {base.py:73} INFO - Using connection ID 'sqlite_default' for task execution.
[2022-12-23 02:20:18,708] {taskinstance.py:1322} INFO - Marking task as SUCCESS. dag_id=calculate_popular_movies, task_id=top_five_animations, execution_date=20221223T022017, start_date=, end_date=20221223T022018
[2022-12-23 02:20:18,708] {taskinstance.py:1322} INFO - Marking task as SUCCESS. dag_id=calculate_popular_movies, task_id=top_five_animations, execution_date=20221223T022017, start_date=, end_date=20221223T022018
[2022-12-23 02:20:18,711] {dag.py:3626} INFO - top_five_animations ran successfully!
[2022-12-23 02:20:18,711] {dag.py:3629} INFO - *****************************************************
[2022-12-23 02:20:18,713] {dagrun.py:606} INFO - Marking run <DagRun calculate_popular_movies @ 2022-12-23T02:20:17.396648+00:00: manual__2022-12-23T02:20:17.396648+00:00, state:running, queued_at: None. externally triggered: False> successful
[2022-12-23 02:20:18,715] {dagrun.py:657} INFO - DagRun Finished: dag_id=calculate_popular_movies, execution_date=2022-12-23T02:20:17.396648+00:00, run_id=manual__2022-12-23T02:20:17.396648+00:00, run_start_date=2022-12-23 02:20:17.396648+00:00, run_end_date=2022-12-23 02:20:18.713514+00:00, run_duration=1.316866, state=success, external_trigger=False, run_type=manual, data_interval_start=2022-12-23T02:20:17.396648+00:00, data_interval_end=2022-12-23T02:20:17.396648+00:00, dag_hash=None
```

*Before 2*:

```
[2022-12-23 01:55:54,386] {load_file.py:92} INFO - Loading https://raw.githubusercontent.com/astronomer/astro-sdk/main/tests/data/imdb_v2.csv into TempTable(name='_tmp_ztujoeesefaqclout728qnyomrc96suvgsntxnen11z4n40ia9wd99roe', conn_id='sqlite_default', metadata=Metadata(schema=None, database=None), columns=[], temp=True) ...
[2022-12-23 01:55:54,388] {base.py:73} INFO - Using connection ID 'sqlite_default' for task execution.
[2022-12-23 01:55:54,393] {base.py:73} INFO - Using connection ID 'sqlite_default' for task execution.
[2022-12-23 01:55:54,499] {base.py:73} INFO - Using connection ID 'sqlite_default' for task execution.
[2022-12-23 01:55:54,507] {base.py:499} INFO - Loading file(s) with Pandas...
[2022-12-23 01:55:54,606] {base.py:73} INFO - Using connection ID 'sqlite_default' for task execution.
[2022-12-23 01:55:54,658] {load_file.py:124} INFO - Completed loading the data into TempTable(name='_tmp_ztujoeesefaqclout728qnyomrc96suvgsntxnen11z4n40ia9wd99roe', conn_id='sqlite_default', metadata=Metadata(schema=None, database=None), columns=[], temp=True).
[2022-12-23 01:55:54,663] {taskinstance.py:1322} INFO - Marking task as SUCCESS. dag_id=calculate_popular_movies, task_id=imdb_movies, execution_date=20221223T015554, start_date=, end_date=20221223T015554
```

*After 2*:

```
[2022-12-23 01:56:37,620] {load_file.py:92} INFO - Loading https://raw.githubusercontent.com/astronomer/astro-sdk/main/tests/data/imdb_v2.csv into TempTable(name='_tmp_rnagpj5gmps5a3oplvlwvlmv6u918qw21inanpxg2j56lo725mrzgp9jo', conn_id='sqlite_default', metadata=Metadata(schema=None, database=None), columns=[], temp=True) ...
[2022-12-23 01:56:37,621] {base.py:73} INFO - Using connection ID 'sqlite_default' for task execution.
[2022-12-23 01:56:37,625] {base.py:73} INFO - Using connection ID 'sqlite_default' for task execution.
[2022-12-23 01:56:37,730] {base.py:501} INFO - Loading file(s) with Pandas...
[2022-12-23 01:56:37,881] {load_file.py:124} INFO - Completed loading the data into TempTable(name='_tmp_rnagpj5gmps5a3oplvlwvlmv6u918qw21inanpxg2j56lo725mrzgp9jo', conn_id='sqlite_default', metadata=Metadata(schema=None, database=None), columns=[], temp=True).
[2022-12-23 01:56:37,886] {taskinstance.py:1322} INFO - Marking task as SUCCESS. dag_id=calculate_popular_movies, task_id=imdb_movies, execution_date=20221223T015637, start_date=, end_date=20221223T015637
```
  • Loading branch information
kaxil authored and utkarsharma2 committed Jan 23, 2023
1 parent d26aa5e commit 8378e76
Show file tree
Hide file tree
Showing 30 changed files with 118 additions and 113 deletions.
2 changes: 1 addition & 1 deletion python-sdk/dev/scripts/pre_commit_context_typing_compat.py
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,7 @@

SOURCES_ROOT = Path(__file__).parents[2]
ASTRO_ROOT = SOURCES_ROOT / "src" / "astro"
TYPING_COMPAT_PATH = "python-sdk/src/astro/utils/typing_compat.py"
TYPING_COMPAT_PATH = "python-sdk/src/astro/utils/compat/typing.py"


class ImportCrawler(NodeVisitor):
Expand Down
3 changes: 2 additions & 1 deletion python-sdk/pyproject.toml
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,8 @@ dependencies = [
"python-frontmatter",
"smart-open",
"SQLAlchemy>=1.3.18",
"apache-airflow-providers-common-sql"
"apache-airflow-providers-common-sql",
"cached_property>=1.5.0;python_version<='3.7'"
]

keywords = ["airflow", "provider", "astronomer", "sql", "decorator", "task flow", "elt", "etl", "dag"]
Expand Down
16 changes: 11 additions & 5 deletions python-sdk/src/astro/databases/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,7 @@
from typing import TYPE_CHECKING

from astro.options import LoadOptionsList
from astro.utils.compat.functools import cache
from astro.utils.path import get_class_name, get_dict_with_module_names_to_dot_notations

if TYPE_CHECKING: # pragma: no cover
Expand Down Expand Up @@ -34,13 +35,18 @@ def create_database(
:param conn_id: Database connection ID in Airflow
:param table: (optional) The Table object
"""
from airflow.hooks.base import BaseHook

conn_type = BaseHook.get_connection(conn_id).conn_type
module_path = CONN_TYPE_TO_MODULE_PATH[conn_type]
module = importlib.import_module(module_path)
module = importlib.import_module(_get_conn(conn_id))
class_name = get_class_name(module_ref=module, suffix="Database")
database_class = getattr(module, class_name)
load_options = load_options_list and load_options_list.get(database_class)
database: BaseDatabase = database_class(conn_id, table, load_options=load_options)
return database


@cache
def _get_conn(conn_id: str) -> str:
from airflow.hooks.base import BaseHook

conn_type = BaseHook.get_connection(conn_id).conn_type
module_path = CONN_TYPE_TO_MODULE_PATH[conn_type]
return module_path
5 changes: 3 additions & 2 deletions python-sdk/src/astro/databases/aws/redshift.py
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
from astro.options import LoadOptions
from astro.settings import REDSHIFT_SCHEMA
from astro.table import BaseTable, Metadata, Table
from astro.utils.compat.functools import cached_property

DEFAULT_CONN_ID = RedshiftSQLHook.default_conn_name
NATIVE_PATHS_SUPPORTED_FILE_TYPES = {
Expand Down Expand Up @@ -89,7 +90,7 @@ def __init__(
def sql_type(self):
return "redshift"

@property
@cached_property
def hook(self) -> RedshiftSQLHook:
"""Retrieve Airflow hook to interface with the Redshift database."""
kwargs = {}
Expand All @@ -100,7 +101,7 @@ def hook(self) -> RedshiftSQLHook:
kwargs.update({"schema": self.table.metadata.database})
return RedshiftSQLHook(redshift_conn_id=self.conn_id, use_legacy_sql=False, **kwargs)

@property
@cached_property
def sqlalchemy_engine(self) -> Engine:
"""Return SQAlchemy engine."""
uri = self.hook.get_uri()
Expand Down
5 changes: 3 additions & 2 deletions python-sdk/src/astro/databases/base.py
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@
from astro.options import LoadOptions
from astro.settings import LOAD_FILE_ENABLE_NATIVE_FALLBACK, LOAD_TABLE_AUTODETECT_ROWS_COUNT, SCHEMA
from astro.table import BaseTable, Metadata
from astro.utils.compat.functools import cached_property


class BaseDatabase(ABC):
Expand Down Expand Up @@ -85,7 +86,7 @@ def __repr__(self):
def sql_type(self):
raise NotImplementedError

@property
@cached_property
def hook(self) -> DbApiHook:
"""Return an instance of the database-specific Airflow hook."""
raise NotImplementedError
Expand All @@ -95,7 +96,7 @@ def connection(self) -> sqlalchemy.engine.base.Connection:
"""Return a Sqlalchemy connection object for the given database."""
return self.sqlalchemy_engine.connect()

@property
@cached_property
def sqlalchemy_engine(self) -> sqlalchemy.engine.base.Engine:
"""Return Sqlalchemy engine."""
return self.hook.get_sqlalchemy_engine() # type: ignore[no-any-return]
Expand Down
5 changes: 3 additions & 2 deletions python-sdk/src/astro/databases/google/bigquery.py
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@
from astro.options import LoadOptions
from astro.settings import BIGQUERY_SCHEMA, BIGQUERY_SCHEMA_LOCATION
from astro.table import BaseTable, Metadata
from astro.utils.compat.functools import cached_property

DEFAULT_CONN_ID = BigQueryHook.default_conn_name
NATIVE_PATHS_SUPPORTED_FILE_TYPES = {
Expand Down Expand Up @@ -119,12 +120,12 @@ def __init__(
def sql_type(self) -> str:
return "bigquery"

@property
@cached_property
def hook(self) -> BigQueryHook:
"""Retrieve Airflow hook to interface with the BigQuery database."""
return BigQueryHook(gcp_conn_id=self.conn_id, use_legacy_sql=False, location=BIGQUERY_SCHEMA_LOCATION)

@property
@cached_property
def sqlalchemy_engine(self) -> Engine:
"""Return SQAlchemy engine."""
uri = self.hook.get_uri()
Expand Down
3 changes: 2 additions & 1 deletion python-sdk/src/astro/databases/postgres.py
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@
from astro.options import LoadOptions
from astro.settings import POSTGRES_SCHEMA
from astro.table import BaseTable, Metadata
from astro.utils.compat.functools import cached_property

DEFAULT_CONN_ID = PostgresHook.default_conn_name

Expand Down Expand Up @@ -43,7 +44,7 @@ def __init__(
def sql_type(self) -> str:
return "postgresql"

@property
@cached_property
def hook(self) -> PostgresHook:
"""Retrieve Airflow hook to interface with the Postgres database."""
conn = PostgresHook(postgres_conn_id=self.conn_id).get_connection(self.conn_id)
Expand Down
3 changes: 2 additions & 1 deletion python-sdk/src/astro/databases/snowflake.py
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,7 @@
from astro.options import SnowflakeLoadOptions
from astro.settings import LOAD_TABLE_AUTODETECT_ROWS_COUNT, SNOWFLAKE_SCHEMA
from astro.table import BaseTable, Metadata
from astro.utils.compat.functools import cached_property

DEFAULT_CONN_ID = SnowflakeHook.default_conn_name

Expand Down Expand Up @@ -267,7 +268,7 @@ def __init__(
raise ValueError("Error: Requires a SnowflakeLoadOptions")
self.load_options: SnowflakeLoadOptions | None = load_options

@property
@cached_property
def hook(self) -> SnowflakeHook:
"""Retrieve Airflow hook to interface with the snowflake database."""
kwargs = {}
Expand Down
5 changes: 3 additions & 2 deletions python-sdk/src/astro/databases/sqlite.py
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@
from astro.databases.base import BaseDatabase
from astro.options import LoadOptions
from astro.table import BaseTable, Metadata
from astro.utils.compat.functools import cached_property

DEFAULT_CONN_ID = SqliteHook.default_conn_name

Expand All @@ -35,12 +36,12 @@ def __init__(
def sql_type(self) -> str:
return "sqlite"

@property
@cached_property
def hook(self) -> SqliteHook:
"""Retrieve Airflow hook to interface with the Sqlite database."""
return SqliteHook(sqlite_conn_id=self.conn_id)

@property
@cached_property
def sqlalchemy_engine(self) -> Engine:
"""Return SQAlchemy engine."""
# Airflow uses sqlite3 library and not SqlAlchemy for SqliteHook
Expand Down
2 changes: 1 addition & 1 deletion python-sdk/src/astro/files/operators/files.py
Original file line number Diff line number Diff line change
Expand Up @@ -7,7 +7,7 @@

from astro.files.base import File
from astro.files.locations import create_file_location
from astro.utils.typing_compat import Context
from astro.utils.compat.typing import Context


class ListFileOperator(BaseOperator):
Expand Down
2 changes: 1 addition & 1 deletion python-sdk/src/astro/sql/operators/append.py
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,7 @@
from astro.databases import create_database
from astro.sql.operators.base_operator import AstroSQLBaseOperator
from astro.table import BaseTable
from astro.utils.typing_compat import Context
from astro.utils.compat.typing import Context


class AppendOperator(AstroSQLBaseOperator):
Expand Down
2 changes: 1 addition & 1 deletion python-sdk/src/astro/sql/operators/base_decorator.py
Original file line number Diff line number Diff line change
Expand Up @@ -13,8 +13,8 @@
from astro.databases.base import BaseDatabase
from astro.sql.operators.upstream_task_mixin import UpstreamTaskMixin
from astro.table import BaseTable, Table
from astro.utils.compat.typing import Context
from astro.utils.table import find_first_table
from astro.utils.typing_compat import Context


class BaseSQLDecoratedOperator(UpstreamTaskMixin, DecoratedOperator):
Expand Down
2 changes: 1 addition & 1 deletion python-sdk/src/astro/sql/operators/cleanup.py
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@
from astro.sql.operators.dataframe import DataframeOperator
from astro.sql.operators.load_file import LoadFileOperator
from astro.table import BaseTable, TempTable
from astro.utils.typing_compat import Context
from astro.utils.compat.typing import Context

OPERATOR_CLASSES_WITH_TABLE_OUTPUT = (
DataframeOperator,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,7 @@

from astro.databases import create_database
from astro.table import BaseTable
from astro.utils.typing_compat import Context
from astro.utils.compat.typing import Context


class ColumnCheckOperator(SQLColumnCheckOperator):
Expand Down
2 changes: 1 addition & 1 deletion python-sdk/src/astro/sql/operators/dataframe.py
Original file line number Diff line number Diff line change
Expand Up @@ -21,9 +21,9 @@
from astro.files import File
from astro.sql.operators.base_operator import AstroSQLBaseOperator
from astro.sql.table import BaseTable, Table
from astro.utils.compat.typing import Context
from astro.utils.dataframe import convert_columns_names_capitalization
from astro.utils.table import find_first_table
from astro.utils.typing_compat import Context


def _get_dataframe(
Expand Down
2 changes: 1 addition & 1 deletion python-sdk/src/astro/sql/operators/drop.py
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,7 @@
from astro.databases import create_database
from astro.sql.operators.base_operator import AstroSQLBaseOperator
from astro.table import BaseTable
from astro.utils.typing_compat import Context
from astro.utils.compat.typing import Context


class DropTableOperator(AstroSQLBaseOperator):
Expand Down
2 changes: 1 addition & 1 deletion python-sdk/src/astro/sql/operators/export_to_file.py
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,7 @@
from astro.files import File
from astro.sql.operators.base_operator import AstroSQLBaseOperator
from astro.table import BaseTable, Table
from astro.utils.typing_compat import Context
from astro.utils.compat.typing import Context


class ExportToFileOperator(AstroSQLBaseOperator):
Expand Down
2 changes: 1 addition & 1 deletion python-sdk/src/astro/sql/operators/load_file.py
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
from astro.settings import LOAD_FILE_ENABLE_NATIVE_FALLBACK
from astro.sql.operators.base_operator import AstroSQLBaseOperator
from astro.table import BaseTable
from astro.utils.typing_compat import Context
from astro.utils.compat.typing import Context


class LoadFileOperator(AstroSQLBaseOperator):
Expand Down
2 changes: 1 addition & 1 deletion python-sdk/src/astro/sql/operators/merge.py
Original file line number Diff line number Diff line change
Expand Up @@ -10,7 +10,7 @@
from astro.databases import create_database
from astro.sql.operators.base_operator import AstroSQLBaseOperator
from astro.table import BaseTable
from astro.utils.typing_compat import Context
from astro.utils.compat.typing import Context


class MergeOperator(AstroSQLBaseOperator):
Expand Down
2 changes: 1 addition & 1 deletion python-sdk/src/astro/sql/operators/raw_sql.py
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@
from astro import settings
from astro.exceptions import IllegalLoadToDatabaseException
from astro.sql.operators.base_decorator import BaseSQLDecoratedOperator
from astro.utils.typing_compat import Context
from astro.utils.compat.typing import Context


class RawSQLOperator(BaseSQLDecoratedOperator):
Expand Down
6 changes: 4 additions & 2 deletions python-sdk/src/astro/sql/operators/transform.py
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@
from sqlalchemy.sql.functions import Function

from astro.sql.operators.base_decorator import BaseSQLDecoratedOperator
from astro.utils.typing_compat import Context
from astro.utils.compat.typing import Context


class TransformOperator(BaseSQLDecoratedOperator):
Expand Down Expand Up @@ -60,7 +60,9 @@ def execute(self, context: Context):
parameters=self.parameters,
)
# TODO: remove pushing to XCom once we update the airflow version.
context["ti"].xcom_push(key="output_table_row_count", value=str(self.output_table.row_count))
context["ti"].xcom_push(
key="output_table_row_count", value=str(self.database_impl.row_count(self.output_table))
)
context["ti"].xcom_push(key="output_table_conn_id", value=str(self.output_table.conn_id))
return self.output_table

Expand Down
Empty file.
18 changes: 18 additions & 0 deletions python-sdk/src/astro/utils/compat/functools.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,18 @@
from __future__ import annotations

import sys

if sys.version_info >= (3, 8):
from functools import cached_property
else:
from cached_property import cached_property

if sys.version_info >= (3, 9):
from functools import cache
else:
from functools import lru_cache

cache = lru_cache(maxsize=None)


__all__ = ["cache", "cached_property"]
File renamed without changes.
2 changes: 1 addition & 1 deletion python-sdk/src/astro/utils/table.py
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,7 @@
from airflow.models.xcom_arg import XComArg

from astro.sql.table import BaseTable
from astro.utils.typing_compat import Context
from astro.utils.compat.typing import Context


def _have_same_conn_id(tables: list[BaseTable]) -> bool:
Expand Down
19 changes: 19 additions & 0 deletions python-sdk/tests/databases/test_bigquery.py
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@

from astro import settings
from astro.databases.google.bigquery import BigqueryDatabase, S3ToBigqueryDataTransfer
from astro.exceptions import DatabaseCustomError
from astro.files import File
from astro.table import TEMP_PREFIX, Metadata, Table

Expand Down Expand Up @@ -139,3 +140,21 @@ def mock_get_dataset(dataset_id):

db = BigqueryDatabase(table=source_table, conn_id="test_conn")
assert db.populate_table_metadata(input_table) == returned_table


@mock.patch("astro.databases.google.bigquery.BigqueryDatabase.hook")
def test_get_project_id_raise_exception(mock_hook):
"""
Test loading on files to bigquery natively for fallback without fallback
gracefully for wrong file location.
"""

class CustomAttributeError:
def __str__(self):
raise AttributeError

database = BigqueryDatabase()
mock_hook.project_id = CustomAttributeError()

with pytest.raises(DatabaseCustomError):
database.get_project_id(target_table=Table())
Loading

0 comments on commit 8378e76

Please sign in to comment.