-
Notifications
You must be signed in to change notification settings - Fork 479
Expand file tree
/
Copy pathtest_sql.py
More file actions
346 lines (275 loc) · 12.7 KB
/
test_sql.py
File metadata and controls
346 lines (275 loc) · 12.7 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
from collections.abc import Generator
from pathlib import Path
from typing import cast
import pytest
from sqlalchemy import Engine, create_engine, inspect, text
from sqlalchemy.exc import ArgumentError
from pyiceberg.catalog import load_catalog
from pyiceberg.catalog.sql import (
DEFAULT_ECHO_VALUE,
DEFAULT_POOL_PRE_PING_VALUE,
IcebergTables,
SqlCatalog,
SqlCatalogBaseTable,
)
from pyiceberg.exceptions import (
NoSuchPropertyException,
TableAlreadyExistsError,
)
from pyiceberg.schema import Schema
from pyiceberg.types import NestedField, StringType, strtobool
CATALOG_TABLES = [c.__tablename__ for c in SqlCatalogBaseTable.__subclasses__()]
@pytest.fixture(scope="module")
def catalog_name() -> str:
return "test_sql_catalog"
@pytest.fixture(scope="module")
def catalog_memory(catalog_name: str, warehouse: Path) -> Generator[SqlCatalog, None, None]:
props = {
"uri": "sqlite:///:memory:",
"warehouse": f"file://{warehouse}",
}
catalog = SqlCatalog(catalog_name, **props)
catalog.create_tables()
yield catalog
catalog.destroy_tables()
@pytest.fixture(scope="module")
def catalog_sqlite(catalog_name: str, warehouse: Path) -> Generator[SqlCatalog, None, None]:
props = {
"uri": f"sqlite:////{warehouse}/sql-catalog",
"warehouse": f"file://{warehouse}",
}
catalog = SqlCatalog(catalog_name, **props)
catalog.create_tables()
yield catalog
catalog.destroy_tables()
@pytest.fixture(scope="module")
def catalog_uri(warehouse: Path) -> str:
return f"sqlite:////{warehouse}/sql-catalog"
@pytest.fixture(scope="module")
def alchemy_engine(catalog_uri: str) -> Engine:
return create_engine(catalog_uri)
def test_creation_with_no_uri(catalog_name: str) -> None:
with pytest.raises(NoSuchPropertyException):
SqlCatalog(catalog_name, not_uri="unused")
def test_creation_with_unsupported_uri(catalog_name: str) -> None:
with pytest.raises(ArgumentError):
SqlCatalog(catalog_name, uri="unsupported:xxx")
def test_creation_with_echo_parameter(catalog_name: str, warehouse: Path) -> None:
# echo_param, expected_echo_value
test_cases = [(None, strtobool(DEFAULT_ECHO_VALUE)), ("debug", "debug"), ("true", True), ("false", False)]
for echo_param, expected_echo_value in test_cases:
props = {
"uri": f"sqlite:////{warehouse}/sql-catalog",
"warehouse": f"file://{warehouse}",
}
# None is for default value
if echo_param is not None:
props["echo"] = echo_param
catalog = SqlCatalog(catalog_name, **props)
assert catalog.engine._echo == expected_echo_value, (
f"Assertion failed: expected echo value {expected_echo_value}, "
f"but got {catalog.engine._echo}. For echo_param={echo_param}"
)
def test_creation_with_pool_pre_ping_parameter(catalog_name: str, warehouse: Path) -> None:
# pool_pre_ping_param, expected_pool_pre_ping_value
test_cases = [
(None, strtobool(DEFAULT_POOL_PRE_PING_VALUE)),
("true", True),
("false", False),
]
for pool_pre_ping_param, expected_pool_pre_ping_value in test_cases:
props = {
"uri": f"sqlite:////{warehouse}/sql-catalog",
"warehouse": f"file://{warehouse}",
}
# None is for default value
if pool_pre_ping_param is not None:
props["pool_pre_ping"] = pool_pre_ping_param
catalog = SqlCatalog(catalog_name, **props)
assert catalog.engine.pool._pre_ping == expected_pool_pre_ping_value, (
f"Assertion failed: expected pool_pre_ping value {expected_pool_pre_ping_value}, "
f"but got {catalog.engine.pool._pre_ping}. For pool_pre_ping_param={pool_pre_ping_param}"
)
def test_creation_from_impl(catalog_name: str, warehouse: Path) -> None:
assert isinstance(
load_catalog(
catalog_name,
**{
"py-catalog-impl": "pyiceberg.catalog.sql.SqlCatalog",
"uri": f"sqlite:////{warehouse}/sql-catalog",
"warehouse": f"file://{warehouse}",
},
),
SqlCatalog,
)
def confirm_no_tables_exist(alchemy_engine: Engine) -> None:
inspector = inspect(alchemy_engine)
for c in SqlCatalogBaseTable.__subclasses__():
if inspector.has_table(c.__tablename__):
c.__table__.drop(alchemy_engine)
any_table_exists = any(t for t in inspector.get_table_names() if t in CATALOG_TABLES)
if any_table_exists:
pytest.raises(TableAlreadyExistsError, "Tables exist, but should not have been created yet")
def confirm_all_tables_exist(catalog: SqlCatalog) -> None:
all_tables_exists = True
for t in CATALOG_TABLES:
if t not in inspect(catalog.engine).get_table_names():
all_tables_exists = False
assert isinstance(catalog, SqlCatalog), "Catalog should be a SQLCatalog"
assert all_tables_exists, "Tables should have been created"
def load_catalog_for_catalog_table_creation(catalog_name: str, catalog_uri: str) -> SqlCatalog:
catalog = load_catalog(
catalog_name,
type="sql",
uri=catalog_uri,
init_catalog_tables="true",
)
return cast(SqlCatalog, catalog)
def test_creation_when_no_tables_exist(alchemy_engine: Engine, catalog_name: str, catalog_uri: str) -> None:
confirm_no_tables_exist(alchemy_engine)
catalog = load_catalog_for_catalog_table_creation(catalog_name=catalog_name, catalog_uri=catalog_uri)
confirm_all_tables_exist(catalog)
def test_creation_when_one_tables_exists(alchemy_engine: Engine, catalog_name: str, catalog_uri: str) -> None:
confirm_no_tables_exist(alchemy_engine)
# Create one table
inspector = inspect(alchemy_engine)
IcebergTables.__table__.create(bind=alchemy_engine)
assert IcebergTables.__tablename__ in [t for t in inspector.get_table_names() if t in CATALOG_TABLES]
catalog = load_catalog_for_catalog_table_creation(catalog_name=catalog_name, catalog_uri=catalog_uri)
confirm_all_tables_exist(catalog)
def test_creation_when_all_tables_exists(alchemy_engine: Engine, catalog_name: str, catalog_uri: str) -> None:
confirm_no_tables_exist(alchemy_engine)
# Create all tables
inspector = inspect(alchemy_engine)
SqlCatalogBaseTable.metadata.create_all(bind=alchemy_engine)
for c in CATALOG_TABLES:
assert c in [t for t in inspector.get_table_names() if t in CATALOG_TABLES]
catalog = load_catalog_for_catalog_table_creation(catalog_name=catalog_name, catalog_uri=catalog_uri)
confirm_all_tables_exist(catalog)
class TestSqlCatalogClose:
"""Test SqlCatalog close functionality."""
def test_sql_catalog_close(self, catalog_sqlite: SqlCatalog) -> None:
"""Test that SqlCatalog close method properly disposes the engine."""
# Verify engine exists
assert hasattr(catalog_sqlite, "engine")
# Close the catalog
catalog_sqlite.close()
# Verify engine is disposed by checking that the engine still exists
assert hasattr(catalog_sqlite, "engine")
def test_sql_catalog_context_manager(self, warehouse: Path) -> None:
"""Test that SqlCatalog works as a context manager."""
with SqlCatalog("test", uri="sqlite:///:memory:", warehouse=str(warehouse)) as catalog:
# Verify engine exists
assert hasattr(catalog, "engine")
# Create a namespace and table to test functionality
catalog.create_namespace("test_db")
schema = Schema(NestedField(1, "name", StringType(), required=True))
catalog.create_table(("test_db", "test_table"), schema)
# Verify engine is disposed after exiting context
assert hasattr(catalog, "engine")
def test_sql_catalog_context_manager_with_exception(self) -> None:
"""Test that SqlCatalog context manager properly closes even with exceptions."""
catalog = None
try:
with SqlCatalog("test", uri="sqlite:///:memory:") as cat:
catalog = cat
# Verify engine exists
assert hasattr(catalog, "engine")
raise ValueError("Test exception")
except ValueError:
pass
# Verify engine is disposed even after exception
assert catalog is not None
assert hasattr(catalog, "engine")
def test_sql_catalog_multiple_close_calls(self, catalog_sqlite: SqlCatalog) -> None:
"""Test that multiple close calls on SqlCatalog are safe."""
# First close
catalog_sqlite.close()
# Second close should not raise an exception
catalog_sqlite.close()
def get_columns(engine: Engine) -> set[str]:
return {c["name"] for c in inspect(engine).get_columns("iceberg_tables")}
def test_adds_iceberg_type_column_to_old_schema(warehouse: Path) -> None:
# Create the old schema tables
uri = f"sqlite:////{warehouse}/test-migration-add-col"
engine = create_engine(uri)
with engine.connect() as conn:
conn.execute(
text(
"CREATE TABLE iceberg_tables ("
" catalog_name VARCHAR(255) NOT NULL,"
" table_namespace VARCHAR(255) NOT NULL,"
" table_name VARCHAR(255) NOT NULL,"
" metadata_location VARCHAR(1000),"
" previous_metadata_location VARCHAR(1000),"
" PRIMARY KEY (catalog_name, table_namespace, table_name)"
")"
)
)
conn.execute(
text(
"CREATE TABLE iceberg_namespace_properties ("
" catalog_name VARCHAR(255) NOT NULL,"
" namespace VARCHAR(255) NOT NULL,"
" property_key VARCHAR(255) NOT NULL,"
" property_value VARCHAR(1000) NOT NULL,"
" PRIMARY KEY (catalog_name, namespace, property_key)"
")"
)
)
conn.commit()
# Verify the column does not exist in the old schema
assert "iceberg_type" not in get_columns(engine)
# Load the catalog and verify the column exists
catalog = SqlCatalog("test", uri=uri, warehouse=f"file://{warehouse}", init_catalog_tables="false")
assert "iceberg_type" in get_columns(catalog.engine)
def test_idempotent_when_column_already_exists(warehouse: Path) -> None:
# Verify the column was created by the init_tables call
catalog = SqlCatalog("test", uri="sqlite:///:memory:", warehouse=f"file://{warehouse}")
assert "iceberg_type" in get_columns(catalog.engine)
# Verify the method is idempotent by calling it again
catalog._update_tables_if_required()
assert "iceberg_type" in get_columns(catalog.engine)
def test_list_tables_filters_by_iceberg_type(warehouse: Path) -> None:
catalog = SqlCatalog("test", uri="sqlite:///:memory:", warehouse=f"file://{warehouse}")
catalog.create_namespace("ns")
schema = Schema(NestedField(1, "id", StringType(), required=True))
catalog.create_table(("ns", "table_V1"), schema)
# Insert a legac-schema row (iceberg_type IS NULL), so itshould appear in list_tables
with catalog.engine.connect() as conn:
conn.execute(
text(
"INSERT INTO iceberg_tables "
"(catalog_name, table_namespace, table_name, metadata_location, previous_metadata_location, iceberg_type) "
"VALUES ('test', 'ns', 'table_V0', NULL, NULL, NULL)"
)
)
# Insert a non-TABLE row — should NOT appear in list_tables
conn.execute(
text(
"INSERT INTO iceberg_tables "
"(catalog_name, table_namespace, table_name, metadata_location, previous_metadata_location, iceberg_type) "
"VALUES ('test', 'ns', 'some_view', NULL, NULL, 'VIEW')"
)
)
conn.commit()
tables = [t[-1] for t in catalog.list_tables("ns")]
assert "table_V1" in tables
assert "table_V0" in tables
assert "some_view" not in tables