-
Notifications
You must be signed in to change notification settings - Fork 28k
/
catalog.py
351 lines (280 loc) · 12 KB
/
catalog.py
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
347
348
349
350
351
#
# 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 pyspark.sql.connect.utils import check_dependencies
check_dependencies(__name__)
from typing import Any, Callable, List, Optional, TYPE_CHECKING
import warnings
import pandas as pd
from pyspark.storagelevel import StorageLevel
from pyspark.sql.types import StructType
from pyspark.sql.connect.dataframe import DataFrame
from pyspark.sql.catalog import (
Catalog as PySparkCatalog,
CatalogMetadata,
Database,
Table,
Function,
Column,
)
from pyspark.sql.connect import plan
if TYPE_CHECKING:
from pyspark.sql.connect.session import SparkSession
from pyspark.sql.connect._typing import DataTypeOrString, UserDefinedFunctionLike
class Catalog:
def __init__(self, sparkSession: "SparkSession") -> None:
self._sparkSession = sparkSession
def _execute_and_fetch(self, catalog: plan.LogicalPlan) -> pd.DataFrame:
pdf = DataFrame.withPlan(catalog, session=self._sparkSession).toPandas()
assert pdf is not None
return pdf
def currentCatalog(self) -> str:
pdf = self._execute_and_fetch(plan.CurrentCatalog())
assert pdf is not None
return pdf.iloc[0].iloc[0]
currentCatalog.__doc__ = PySparkCatalog.currentCatalog.__doc__
def setCurrentCatalog(self, catalogName: str) -> None:
self._execute_and_fetch(plan.SetCurrentCatalog(catalog_name=catalogName))
setCurrentCatalog.__doc__ = PySparkCatalog.setCurrentCatalog.__doc__
def listCatalogs(self, pattern: Optional[str] = None) -> List[CatalogMetadata]:
pdf = self._execute_and_fetch(plan.ListCatalogs(pattern=pattern))
return [
CatalogMetadata(name=row.iloc[0], description=row.iloc[1]) for _, row in pdf.iterrows()
]
listCatalogs.__doc__ = PySparkCatalog.listCatalogs.__doc__
def currentDatabase(self) -> str:
pdf = self._execute_and_fetch(plan.CurrentDatabase())
assert pdf is not None
return pdf.iloc[0].iloc[0]
currentDatabase.__doc__ = PySparkCatalog.currentDatabase.__doc__
def setCurrentDatabase(self, dbName: str) -> None:
self._execute_and_fetch(plan.SetCurrentDatabase(db_name=dbName))
setCurrentDatabase.__doc__ = PySparkCatalog.setCurrentDatabase.__doc__
def listDatabases(self, pattern: Optional[str] = None) -> List[Database]:
pdf = self._execute_and_fetch(plan.ListDatabases(pattern=pattern))
return [
Database(
name=row.iloc[0],
catalog=row.iloc[1],
description=row.iloc[2],
locationUri=row.iloc[3],
)
for _, row in pdf.iterrows()
]
listDatabases.__doc__ = PySparkCatalog.listDatabases.__doc__
def getDatabase(self, dbName: str) -> Database:
pdf = self._execute_and_fetch(plan.GetDatabase(db_name=dbName))
assert pdf is not None
row = pdf.iloc[0]
return Database(
name=row[0],
catalog=row[1],
description=row[2],
locationUri=row[3],
)
getDatabase.__doc__ = PySparkCatalog.getDatabase.__doc__
def databaseExists(self, dbName: str) -> bool:
pdf = self._execute_and_fetch(plan.DatabaseExists(db_name=dbName))
assert pdf is not None
return pdf.iloc[0].iloc[0]
databaseExists.__doc__ = PySparkCatalog.databaseExists.__doc__
def listTables(
self, dbName: Optional[str] = None, pattern: Optional[str] = None
) -> List[Table]:
pdf = self._execute_and_fetch(plan.ListTables(db_name=dbName, pattern=pattern))
return [
Table(
name=row.iloc[0],
catalog=row.iloc[1],
# If None, returns None.
namespace=None if row.iloc[2] is None else list(row.iloc[2]),
description=row.iloc[3],
tableType=row.iloc[4],
isTemporary=row.iloc[5],
)
for _, row in pdf.iterrows()
]
listTables.__doc__ = PySparkCatalog.listTables.__doc__
def getTable(self, tableName: str) -> Table:
pdf = self._execute_and_fetch(plan.GetTable(table_name=tableName))
assert pdf is not None
row = pdf.iloc[0]
return Table(
name=row.iloc[0],
catalog=row.iloc[1],
# If None, returns None.
namespace=None if row.iloc[2] is None else list(row.iloc[2]),
description=row.iloc[3],
tableType=row.iloc[4],
isTemporary=row.iloc[5],
)
getTable.__doc__ = PySparkCatalog.getTable.__doc__
def listFunctions(self, dbName: Optional[str] = None) -> List[Function]:
pdf = self._execute_and_fetch(plan.ListFunctions(db_name=dbName))
return [
Function(
name=row.iloc[0],
catalog=row.iloc[1],
# If None, returns None.
namespace=None if row.iloc[2] is None else list(row.iloc[2]),
description=row.iloc[3],
className=row.iloc[4],
isTemporary=row.iloc[5],
)
for _, row in pdf.iterrows()
]
listFunctions.__doc__ = PySparkCatalog.listFunctions.__doc__
def functionExists(self, functionName: str, dbName: Optional[str] = None) -> bool:
pdf = self._execute_and_fetch(
plan.FunctionExists(function_name=functionName, db_name=dbName)
)
assert pdf is not None
return pdf.iloc[0].iloc[0]
functionExists.__doc__ = PySparkCatalog.functionExists.__doc__
def getFunction(self, functionName: str) -> Function:
pdf = self._execute_and_fetch(plan.GetFunction(function_name=functionName))
assert pdf is not None
row = pdf.iloc[0]
return Function(
name=row.iloc[0],
catalog=row.iloc[1],
# If None, returns None.
namespace=None if row.iloc[2] is None else list(row.iloc[2]),
description=row.iloc[3],
className=row.iloc[4],
isTemporary=row.iloc[5],
)
getFunction.__doc__ = PySparkCatalog.getFunction.__doc__
def listColumns(self, tableName: str, dbName: Optional[str] = None) -> List[Column]:
pdf = self._execute_and_fetch(plan.ListColumns(table_name=tableName, db_name=dbName))
return [
Column(
name=row.iloc[0],
description=row.iloc[1],
dataType=row.iloc[2],
nullable=row.iloc[3],
isPartition=row.iloc[4],
isBucket=row.iloc[5],
)
for _, row in pdf.iterrows()
]
listColumns.__doc__ = PySparkCatalog.listColumns.__doc__
def tableExists(self, tableName: str, dbName: Optional[str] = None) -> bool:
pdf = self._execute_and_fetch(plan.TableExists(table_name=tableName, db_name=dbName))
assert pdf is not None
return pdf.iloc[0].iloc[0]
tableExists.__doc__ = PySparkCatalog.tableExists.__doc__
def createExternalTable(
self,
tableName: str,
path: Optional[str] = None,
source: Optional[str] = None,
schema: Optional[StructType] = None,
**options: str,
) -> DataFrame:
catalog = plan.CreateExternalTable(
table_name=tableName,
path=path, # type: ignore[arg-type]
source=source,
schema=schema,
options=options,
)
df = DataFrame.withPlan(catalog, session=self._sparkSession)
df.toPandas() # Eager execution.
return df
createExternalTable.__doc__ = PySparkCatalog.createExternalTable.__doc__
def createTable(
self,
tableName: str,
path: Optional[str] = None,
source: Optional[str] = None,
schema: Optional[StructType] = None,
description: Optional[str] = None,
**options: str,
) -> DataFrame:
catalog = plan.CreateTable(
table_name=tableName,
path=path, # type: ignore[arg-type]
source=source,
schema=schema,
description=description,
options=options,
)
df = DataFrame.withPlan(catalog, session=self._sparkSession)
df.toPandas() # Eager execution.
return df
createTable.__doc__ = PySparkCatalog.createTable.__doc__
def dropTempView(self, viewName: str) -> bool:
pdf = self._execute_and_fetch(plan.DropTempView(view_name=viewName))
assert pdf is not None
return pdf.iloc[0].iloc[0]
dropTempView.__doc__ = PySparkCatalog.dropTempView.__doc__
def dropGlobalTempView(self, viewName: str) -> bool:
pdf = self._execute_and_fetch(plan.DropGlobalTempView(view_name=viewName))
assert pdf is not None
return pdf.iloc[0].iloc[0]
dropGlobalTempView.__doc__ = PySparkCatalog.dropGlobalTempView.__doc__
def isCached(self, tableName: str) -> bool:
pdf = self._execute_and_fetch(plan.IsCached(table_name=tableName))
assert pdf is not None
return pdf.iloc[0].iloc[0]
isCached.__doc__ = PySparkCatalog.isCached.__doc__
def cacheTable(self, tableName: str, storageLevel: Optional[StorageLevel] = None) -> None:
self._execute_and_fetch(plan.CacheTable(table_name=tableName, storage_level=storageLevel))
cacheTable.__doc__ = PySparkCatalog.cacheTable.__doc__
def uncacheTable(self, tableName: str) -> None:
self._execute_and_fetch(plan.UncacheTable(table_name=tableName))
uncacheTable.__doc__ = PySparkCatalog.uncacheTable.__doc__
def clearCache(self) -> None:
self._execute_and_fetch(plan.ClearCache())
clearCache.__doc__ = PySparkCatalog.clearCache.__doc__
def refreshTable(self, tableName: str) -> None:
self._execute_and_fetch(plan.RefreshTable(table_name=tableName))
refreshTable.__doc__ = PySparkCatalog.refreshTable.__doc__
def recoverPartitions(self, tableName: str) -> None:
self._execute_and_fetch(plan.RecoverPartitions(table_name=tableName))
recoverPartitions.__doc__ = PySparkCatalog.recoverPartitions.__doc__
def refreshByPath(self, path: str) -> None:
self._execute_and_fetch(plan.RefreshByPath(path=path))
refreshByPath.__doc__ = PySparkCatalog.refreshByPath.__doc__
def registerFunction(
self, name: str, f: Callable[..., Any], returnType: Optional["DataTypeOrString"] = None
) -> "UserDefinedFunctionLike":
warnings.warn("Deprecated in 2.3.0. Use spark.udf.register instead.", FutureWarning)
return self._sparkSession.udf.register(name, f, returnType)
registerFunction.__doc__ = PySparkCatalog.registerFunction.__doc__
Catalog.__doc__ = PySparkCatalog.__doc__
def _test() -> None:
import sys
import doctest
from pyspark.sql import SparkSession as PySparkSession
import pyspark.sql.connect.catalog
globs = pyspark.sql.connect.catalog.__dict__.copy()
globs["spark"] = (
PySparkSession.builder.appName("sql.connect.catalog tests").remote("local[4]").getOrCreate()
)
(failure_count, test_count) = doctest.testmod(
pyspark.sql.connect.catalog,
globs=globs,
optionflags=doctest.ELLIPSIS
| doctest.NORMALIZE_WHITESPACE
| doctest.IGNORE_EXCEPTION_DETAIL,
)
globs["spark"].stop()
if failure_count:
sys.exit(-1)
if __name__ == "__main__":
_test()