From f924fc33ab262cd9a8e47f071182d4fb07bd5058 Mon Sep 17 00:00:00 2001 From: Kevin Liu Date: Sat, 4 Oct 2025 11:05:13 -0700 Subject: [PATCH 1/3] remove unused filterwarnings --- pyproject.toml | 2 -- 1 file changed, 2 deletions(-) diff --git a/pyproject.toml b/pyproject.toml index 1d94838081..ba3b5e2421 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -346,8 +346,6 @@ markers = [ # Turns a warning into an error filterwarnings = [ "error", - "ignore:A plugin raised an exception during an old-style hookwrapper teardown.", - "ignore:unclosed Date: Sat, 4 Oct 2025 12:16:11 -0700 Subject: [PATCH 2/3] fix resource leak --- pyiceberg/catalog/hive.py | 5 +++++ tests/catalog/test_hive.py | 9 +++++++++ 2 files changed, 14 insertions(+) diff --git a/pyiceberg/catalog/hive.py b/pyiceberg/catalog/hive.py index e558c8c30f..93ece35cbb 100644 --- a/pyiceberg/catalog/hive.py +++ b/pyiceberg/catalog/hive.py @@ -185,6 +185,11 @@ def __enter__(self) -> Client: try: self._transport.open() except (TypeError, TTransport.TTransportException): + # Close the old transport before reinitializing to prevent resource leaks + try: + self._transport.close() + except Exception: + pass # reinitialize _transport self._transport = self._init_thrift_transport() self._transport.open() diff --git a/tests/catalog/test_hive.py b/tests/catalog/test_hive.py index 2da3f99f3e..b0f600fd91 100644 --- a/tests/catalog/test_hive.py +++ b/tests/catalog/test_hive.py @@ -204,6 +204,7 @@ def __init__(self, socket: thrift.transport.TSocket.TServerSocket, response: byt self._response = response self._port = None self._port_bound = threading.Event() + self._clients: list = [] # Track accepted client connections def run(self) -> None: self._socket.listen() @@ -222,6 +223,7 @@ def run(self) -> None: try: client = self._socket.accept() if client: + self._clients.append(client) # Track the client client.write(self._response) client.flush() except Exception: @@ -233,6 +235,12 @@ def port(self) -> Optional[int]: return self._port def close(self) -> None: + # Close all client connections first + for client in self._clients: + try: + client.close() + except Exception: + pass self._socket.close() @@ -1392,6 +1400,7 @@ def test_create_hive_client_with_kerberos_using_context_manager( with client as open_client: assert open_client._iprot.trans.isOpen() + assert not open_client._iprot.trans.isOpen() # Use the context manager a second time to see if # closing and re-opening work as expected. with client as open_client: From 44b4486fef6be89f40ace88a05dc07276b140f70 Mon Sep 17 00:00:00 2001 From: Kevin Liu Date: Sat, 4 Oct 2025 12:28:46 -0700 Subject: [PATCH 3/3] make lint --- tests/catalog/test_hive.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/catalog/test_hive.py b/tests/catalog/test_hive.py index b0f600fd91..649e2545ba 100644 --- a/tests/catalog/test_hive.py +++ b/tests/catalog/test_hive.py @@ -204,7 +204,7 @@ def __init__(self, socket: thrift.transport.TSocket.TServerSocket, response: byt self._response = response self._port = None self._port_bound = threading.Event() - self._clients: list = [] # Track accepted client connections + self._clients: list[thrift.transport.TSocket.TSocket] = [] # Track accepted client connections def run(self) -> None: self._socket.listen()