Skip to content

Commit f0b4003

Browse files
committed
re-init transport
1 parent 1407338 commit f0b4003

File tree

1 file changed

+15
-26
lines changed

1 file changed

+15
-26
lines changed

pyiceberg/catalog/hive.py

Lines changed: 15 additions & 26 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@
1818
import logging
1919
import socket
2020
import time
21+
from functools import cached_property
2122
from types import TracebackType
2223
from typing import (
2324
TYPE_CHECKING,
@@ -142,52 +143,40 @@
142143
class _HiveClient:
143144
"""Helper class to nicely open and close the transport."""
144145

145-
_transport: TTransport
146-
_client: Client
147-
_ugi: Optional[List[str]]
148-
149146
def __init__(self, uri: str, ugi: Optional[str] = None, kerberos_auth: Optional[bool] = HIVE_KERBEROS_AUTH_DEFAULT):
150147
self._uri = uri
151148
self._kerberos_auth = kerberos_auth
152149
self._ugi = ugi.split(":") if ugi else None
153150

154-
self._init_thrift_client()
155-
156-
def _init_thrift_client(self) -> None:
151+
def _init_thrift_transport(self) -> TTransport:
157152
url_parts = urlparse(self._uri)
158-
159153
socket = TSocket.TSocket(url_parts.hostname, url_parts.port)
160-
161154
if not self._kerberos_auth:
162-
self._transport = TTransport.TBufferedTransport(socket)
155+
return TTransport.TBufferedTransport(socket)
163156
else:
164-
self._transport = TTransport.TSaslClientTransport(socket, host=url_parts.hostname, service="hive")
157+
return TTransport.TSaslClientTransport(socket, host=url_parts.hostname, service="hive")
165158

159+
@cached_property
160+
def _client(self) -> Client:
161+
self._transport = self._init_thrift_transport()
166162
protocol = TBinaryProtocol.TBinaryProtocol(self._transport)
167-
168-
self._client = Client(protocol)
169-
170-
def __enter__(self) -> Client:
171-
"""Ensure transport is open before returning the client."""
172-
if self._transport is None or not self._transport.isOpen():
173-
self._init_thrift_client() # Reinitialize transport if closed
174-
175-
if not self._transport.isOpen():
176-
self._transport.open()
177-
163+
client = Client(protocol)
178164
if self._ugi:
179-
self._client.set_ugi(*self._ugi)
165+
client.set_ugi(*self._ugi)
166+
return client
180167

168+
def __enter__(self) -> Client:
169+
"""Reinitialize transport if was closed."""
170+
if self._transport and not self._transport.isOpen():
171+
self._transport = self._init_thrift_transport()
181172
return self._client
182173

183174
def __exit__(
184175
self, exctype: Optional[Type[BaseException]], excinst: Optional[BaseException], exctb: Optional[TracebackType]
185176
) -> None:
186177
"""Close transport if it was opened."""
187-
if self._transport:
178+
if self._transport and self._transport.isOpen():
188179
self._transport.close()
189-
self._transport = None # Reset transport so a new one is created next time
190-
self._client = None
191180

192181

193182
def _construct_hive_storage_descriptor(

0 commit comments

Comments
 (0)