Skip to content

Commit 23fd19d

Browse files
committed
fix hive client
1 parent 6793e64 commit 23fd19d

File tree

2 files changed

+4
-5
lines changed

2 files changed

+4
-5
lines changed

pyiceberg/catalog/hive.py

Lines changed: 2 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,6 @@
1818
import logging
1919
import socket
2020
import time
21-
from functools import cached_property
2221
from types import TracebackType
2322
from typing import (
2423
TYPE_CHECKING,
@@ -160,7 +159,6 @@ def _init_thrift_transport(self) -> TTransport:
160159
else:
161160
return TTransport.TSaslClientTransport(socket, host=url_parts.hostname, service="hive")
162161

163-
@cached_property
164162
def _client(self) -> Client:
165163
protocol = TBinaryProtocol.TBinaryProtocol(self._transport)
166164
client = Client(protocol)
@@ -173,11 +171,11 @@ def __enter__(self) -> Client:
173171
if not self._transport.isOpen():
174172
try:
175173
self._transport.open()
176-
except TTransport.TTransportException:
174+
except (TypeError, TTransport.TTransportException):
177175
# reinitialize _transport
178176
self._transport = self._init_thrift_transport()
179177
self._transport.open()
180-
return self._client
178+
return self._client() # recreate the client
181179

182180
def __exit__(
183181
self, exctype: Optional[Type[BaseException]], excinst: Optional[BaseException], exctb: Optional[TracebackType]

tests/catalog/test_hive.py

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -45,6 +45,7 @@
4545

4646
from pyiceberg.catalog import PropertiesUpdateSummary
4747
from pyiceberg.catalog.hive import (
48+
HIVE_KERBEROS_AUTH,
4849
LOCK_CHECK_MAX_WAIT_TIME,
4950
LOCK_CHECK_MIN_WAIT_TIME,
5051
LOCK_CHECK_RETRIES,
@@ -1310,7 +1311,7 @@ def test_create_hive_client_with_kerberos(
13101311
properties = {
13111312
"uri": kerberized_hive_metastore_fake_url,
13121313
"ugi": "user",
1313-
"kerberos_auth": True,
1314+
HIVE_KERBEROS_AUTH: "true",
13141315
}
13151316
client = HiveCatalog._create_hive_client(properties)
13161317
assert client is not None

0 commit comments

Comments
 (0)