Skip to content

Commit

Permalink
Suppress JException when get_autocommit and set_autocommit methods ar…
Browse files Browse the repository at this point in the history
…en't supported on JDBC driver (#43786)

* refactor: get_autocommit and set_autocommit should also suppress JExceptions if those operations aren't supported by the JDBC driver

* refactor: Use the cached connection property instead of get_connection in sqlalchemy_url and get_conn methods

* refactor: Bumped version of jdbc provider to 4.5.3

* Update providers/src/airflow/providers/jdbc/provider.yaml

---------

Co-authored-by: David Blain <[email protected]>
Co-authored-by: Jarek Potiuk <[email protected]>
  • Loading branch information
3 people authored Nov 28, 2024
1 parent 60a38e1 commit b32ca97
Showing 1 changed file with 5 additions and 4 deletions.
9 changes: 5 additions & 4 deletions providers/src/airflow/providers/jdbc/hooks/jdbc.py
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
from typing import TYPE_CHECKING, Any

import jaydebeapi
import jpype
from sqlalchemy.engine import URL

from airflow.exceptions import AirflowException
Expand Down Expand Up @@ -148,7 +149,7 @@ def driver_class(self) -> str | None:

@property
def sqlalchemy_url(self) -> URL:
conn = self.get_connection(getattr(self, self.conn_name_attr))
conn = self.connection
sqlalchemy_scheme = conn.extra_dejson.get("sqlalchemy_scheme")
if sqlalchemy_scheme is None:
raise AirflowException(
Expand Down Expand Up @@ -177,7 +178,7 @@ def get_sqlalchemy_engine(self, engine_kwargs=None):
return super().get_sqlalchemy_engine(engine_kwargs)

def get_conn(self) -> jaydebeapi.Connection:
conn: Connection = self.get_connection(self.get_conn_id())
conn: Connection = self.connection
host: str = conn.host
login: str = conn.login
psw: str = conn.password
Expand All @@ -197,7 +198,7 @@ def set_autocommit(self, conn: jaydebeapi.Connection, autocommit: bool) -> None:
:param conn: The connection.
:param autocommit: The connection's autocommit setting.
"""
with suppress_and_warn(jaydebeapi.Error):
with suppress_and_warn(jaydebeapi.Error, jpype.JException):
conn.jconn.setAutoCommit(autocommit)

def get_autocommit(self, conn: jaydebeapi.Connection) -> bool:
Expand All @@ -209,6 +210,6 @@ def get_autocommit(self, conn: jaydebeapi.Connection) -> bool:
to True on the connection. False if it is either not set, set to
False, or the connection does not support auto-commit.
"""
with suppress_and_warn(jaydebeapi.Error):
with suppress_and_warn(jaydebeapi.Error, jpype.JException):
return conn.jconn.getAutoCommit()
return False

0 comments on commit b32ca97

Please sign in to comment.