-
Notifications
You must be signed in to change notification settings - Fork 28.3k
/
catalog.py
1271 lines (1013 loc) · 39.4 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
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
#
# 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.
#
import sys
import warnings
from typing import Any, Callable, NamedTuple, List, Optional, TYPE_CHECKING
from pyspark.storagelevel import StorageLevel
from pyspark.sql.dataframe import DataFrame
from pyspark.sql.session import SparkSession
from pyspark.sql.types import StructType
if TYPE_CHECKING:
from pyspark.sql._typing import UserDefinedFunctionLike
from pyspark.sql.types import DataType
class CatalogMetadata(NamedTuple):
name: str
description: Optional[str]
class Database(NamedTuple):
name: str
catalog: Optional[str]
description: Optional[str]
locationUri: str
class Table(NamedTuple):
name: str
catalog: Optional[str]
namespace: Optional[List[str]]
description: Optional[str]
tableType: str
isTemporary: bool
@property
def database(self) -> Optional[str]:
if self.namespace is not None and len(self.namespace) == 1:
return self.namespace[0]
else:
return None
class Column(NamedTuple):
name: str
description: Optional[str]
dataType: str
nullable: bool
isPartition: bool
isBucket: bool
class Function(NamedTuple):
name: str
catalog: Optional[str]
namespace: Optional[List[str]]
description: Optional[str]
className: str
isTemporary: bool
class Catalog:
"""User-facing catalog API, accessible through `SparkSession.catalog`.
This is a thin wrapper around its Scala implementation org.apache.spark.sql.catalog.Catalog.
.. versionchanged:: 3.4.0
Supports Spark Connect.
"""
def __init__(self, sparkSession: SparkSession) -> None:
"""Create a new Catalog that wraps the underlying JVM object."""
self._sparkSession = sparkSession
self._jsparkSession = sparkSession._jsparkSession
self._sc = sparkSession._sc
self._jcatalog = sparkSession._jsparkSession.catalog()
def currentCatalog(self) -> str:
"""Returns the current default catalog in this session.
.. versionadded:: 3.4.0
Examples
--------
>>> spark.catalog.currentCatalog()
'spark_catalog'
"""
return self._jcatalog.currentCatalog()
def setCurrentCatalog(self, catalogName: str) -> None:
"""Sets the current default catalog in this session.
.. versionadded:: 3.4.0
Parameters
----------
catalogName : str
name of the catalog to set
Examples
--------
>>> spark.catalog.setCurrentCatalog("spark_catalog")
"""
return self._jcatalog.setCurrentCatalog(catalogName)
def listCatalogs(self, pattern: Optional[str] = None) -> List[CatalogMetadata]:
"""Returns a list of catalogs in this session.
.. versionadded:: 3.4.0
Parameters
----------
pattern : str
The pattern that the catalog name needs to match.
.. versionchanged: 3.5.0
Added ``pattern`` argument.
Returns
-------
list
A list of :class:`CatalogMetadata`.
Examples
--------
>>> spark.catalog.listCatalogs()
[CatalogMetadata(name='spark_catalog', description=None)]
>>> spark.catalog.listCatalogs("spark*")
[CatalogMetadata(name='spark_catalog', description=None)]
>>> spark.catalog.listCatalogs("hive*")
[]
"""
if pattern is None:
iter = self._jcatalog.listCatalogs().toLocalIterator()
else:
iter = self._jcatalog.listCatalogs(pattern).toLocalIterator()
catalogs = []
while iter.hasNext():
jcatalog = iter.next()
catalogs.append(
CatalogMetadata(name=jcatalog.name(), description=jcatalog.description())
)
return catalogs
def currentDatabase(self) -> str:
"""
Returns the current default database in this session.
.. versionadded:: 2.0.0
Returns
-------
str
The current default database name.
Examples
--------
>>> spark.catalog.currentDatabase()
'default'
"""
return self._jcatalog.currentDatabase()
def setCurrentDatabase(self, dbName: str) -> None:
"""
Sets the current default database in this session.
.. versionadded:: 2.0.0
Examples
--------
>>> spark.catalog.setCurrentDatabase("default")
"""
return self._jcatalog.setCurrentDatabase(dbName)
def listDatabases(self, pattern: Optional[str] = None) -> List[Database]:
"""
Returns a list of databases available across all sessions.
.. versionadded:: 2.0.0
Parameters
----------
pattern : str
The pattern that the database name needs to match.
.. versionchanged: 3.5.0
Adds ``pattern`` argument.
Returns
-------
list
A list of :class:`Database`.
Examples
--------
>>> spark.catalog.listDatabases()
[Database(name='default', catalog='spark_catalog', description='default database', ...
>>> spark.catalog.listDatabases("def*")
[Database(name='default', catalog='spark_catalog', description='default database', ...
>>> spark.catalog.listDatabases("def2*")
[]
"""
if pattern is None:
iter = self._jcatalog.listDatabases().toLocalIterator()
else:
iter = self._jcatalog.listDatabases(pattern).toLocalIterator()
databases = []
while iter.hasNext():
jdb = iter.next()
databases.append(
Database(
name=jdb.name(),
catalog=jdb.catalog(),
description=jdb.description(),
locationUri=jdb.locationUri(),
)
)
return databases
def getDatabase(self, dbName: str) -> Database:
"""Get the database with the specified name.
This throws an :class:`AnalysisException` when the database cannot be found.
.. versionadded:: 3.4.0
Parameters
----------
dbName : str
name of the database to get.
Returns
-------
:class:`Database`
The database found by the name.
Examples
--------
>>> spark.catalog.getDatabase("default")
Database(name='default', catalog='spark_catalog', description='default database', ...
Using the fully qualified name with the catalog name.
>>> spark.catalog.getDatabase("spark_catalog.default")
Database(name='default', catalog='spark_catalog', description='default database', ...
"""
jdb = self._jcatalog.getDatabase(dbName)
return Database(
name=jdb.name(),
catalog=jdb.catalog(),
description=jdb.description(),
locationUri=jdb.locationUri(),
)
def databaseExists(self, dbName: str) -> bool:
"""Check if the database with the specified name exists.
.. versionadded:: 3.3.0
Parameters
----------
dbName : str
name of the database to check existence
.. versionchanged:: 3.4.0
Allow ``dbName`` to be qualified with catalog name.
Returns
-------
bool
Indicating whether the database exists
Examples
--------
Check if 'test_new_database' database exists
>>> spark.catalog.databaseExists("test_new_database")
False
>>> _ = spark.sql("CREATE DATABASE test_new_database")
>>> spark.catalog.databaseExists("test_new_database")
True
Using the fully qualified name with the catalog name.
>>> spark.catalog.databaseExists("spark_catalog.test_new_database")
True
>>> _ = spark.sql("DROP DATABASE test_new_database")
"""
return self._jcatalog.databaseExists(dbName)
def listTables(
self, dbName: Optional[str] = None, pattern: Optional[str] = None
) -> List[Table]:
"""Returns a list of tables/views in the specified database.
.. versionadded:: 2.0.0
Parameters
----------
dbName : str
name of the database to list the tables.
.. versionchanged:: 3.4.0
Allow ``dbName`` to be qualified with catalog name.
pattern : str
The pattern that the database name needs to match.
.. versionchanged: 3.5.0
Adds ``pattern`` argument.
Returns
-------
list
A list of :class:`Table`.
Notes
-----
If no database is specified, the current database and catalog
are used. This API includes all temporary views.
Examples
--------
>>> spark.range(1).createTempView("test_view")
>>> spark.catalog.listTables()
[Table(name='test_view', catalog=None, namespace=[], description=None, ...
>>> spark.catalog.listTables(pattern="test*")
[Table(name='test_view', catalog=None, namespace=[], description=None, ...
>>> spark.catalog.listTables(pattern="table*")
[]
>>> _ = spark.catalog.dropTempView("test_view")
>>> spark.catalog.listTables()
[]
"""
if dbName is None:
dbName = self.currentDatabase()
if pattern is None:
iter = self._jcatalog.listTables(dbName).toLocalIterator()
else:
iter = self._jcatalog.listTables(dbName, pattern).toLocalIterator()
tables = []
while iter.hasNext():
jtable = iter.next()
jnamespace = jtable.namespace()
if jnamespace is not None:
namespace = [jnamespace[i] for i in range(0, len(jnamespace))]
else:
namespace = None
tables.append(
Table(
name=jtable.name(),
catalog=jtable.catalog(),
namespace=namespace,
description=jtable.description(),
tableType=jtable.tableType(),
isTemporary=jtable.isTemporary(),
)
)
return tables
def getTable(self, tableName: str) -> Table:
"""Get the table or view with the specified name. This table can be a temporary view or a
table/view. This throws an :class:`AnalysisException` when no Table can be found.
.. versionadded:: 3.4.0
Parameters
----------
tableName : str
name of the table to get.
.. versionchanged:: 3.4.0
Allow `tableName` to be qualified with catalog name.
Returns
-------
:class:`Table`
The table found by the name.
Examples
--------
>>> _ = spark.sql("DROP TABLE IF EXISTS tbl1")
>>> _ = spark.sql("CREATE TABLE tbl1 (name STRING, age INT) USING parquet")
>>> spark.catalog.getTable("tbl1")
Table(name='tbl1', catalog='spark_catalog', namespace=['default'], ...
Using the fully qualified name with the catalog name.
>>> spark.catalog.getTable("default.tbl1")
Table(name='tbl1', catalog='spark_catalog', namespace=['default'], ...
>>> spark.catalog.getTable("spark_catalog.default.tbl1")
Table(name='tbl1', catalog='spark_catalog', namespace=['default'], ...
>>> _ = spark.sql("DROP TABLE tbl1")
Throw an analysis exception when the table does not exist.
>>> spark.catalog.getTable("tbl1")
Traceback (most recent call last):
...
AnalysisException: ...
"""
jtable = self._jcatalog.getTable(tableName)
jnamespace = jtable.namespace()
if jnamespace is not None:
namespace = [jnamespace[i] for i in range(0, len(jnamespace))]
else:
namespace = None
return Table(
name=jtable.name(),
catalog=jtable.catalog(),
namespace=namespace,
description=jtable.description(),
tableType=jtable.tableType(),
isTemporary=jtable.isTemporary(),
)
def listFunctions(
self, dbName: Optional[str] = None, pattern: Optional[str] = None
) -> List[Function]:
"""
Returns a list of functions registered in the specified database.
.. versionadded:: 3.4.0
Parameters
----------
dbName : str
name of the database to list the functions.
``dbName`` can be qualified with catalog name.
pattern : str
The pattern that the function name needs to match.
.. versionchanged: 3.5.0
Adds ``pattern`` argument.
Returns
-------
list
A list of :class:`Function`.
Notes
-----
If no database is specified, the current database and catalog
are used. This API includes all temporary functions.
Examples
--------
>>> spark.catalog.listFunctions()
[Function(name=...
>>> spark.catalog.listFunctions(pattern="to_*")
[Function(name=...
>>> spark.catalog.listFunctions(pattern="*not_existing_func*")
[]
"""
if dbName is None:
dbName = self.currentDatabase()
iter = self._jcatalog.listFunctions(dbName).toLocalIterator()
if pattern is None:
iter = self._jcatalog.listFunctions(dbName).toLocalIterator()
else:
iter = self._jcatalog.listFunctions(dbName, pattern).toLocalIterator()
functions = []
while iter.hasNext():
jfunction = iter.next()
jnamespace = jfunction.namespace()
if jnamespace is not None:
namespace = [jnamespace[i] for i in range(0, len(jnamespace))]
else:
namespace = None
functions.append(
Function(
name=jfunction.name(),
catalog=jfunction.catalog(),
namespace=namespace,
description=jfunction.description(),
className=jfunction.className(),
isTemporary=jfunction.isTemporary(),
)
)
return functions
def functionExists(self, functionName: str, dbName: Optional[str] = None) -> bool:
"""Check if the function with the specified name exists.
This can either be a temporary function or a function.
.. versionadded:: 3.3.0
Parameters
----------
functionName : str
name of the function to check existence
.. versionchanged:: 3.4.0
Allow ``functionName`` to be qualified with catalog name
dbName : str, optional
name of the database to check function existence in.
Returns
-------
bool
Indicating whether the function exists
Notes
-----
If no database is specified, the current database and catalog
are used. This API includes all temporary functions.
Examples
--------
>>> spark.catalog.functionExists("count")
True
Using the fully qualified name for function name.
>>> spark.catalog.functionExists("default.unexisting_function")
False
>>> spark.catalog.functionExists("spark_catalog.default.unexisting_function")
False
"""
if dbName is None:
return self._jcatalog.functionExists(functionName)
else:
warnings.warn(
"`dbName` has been deprecated since Spark 3.4 and might be removed in "
"a future version. Use functionExists(`dbName.tableName`) instead.",
FutureWarning,
)
return self._jcatalog.functionExists(dbName, functionName)
def getFunction(self, functionName: str) -> Function:
"""Get the function with the specified name. This function can be a temporary function or a
function. This throws an :class:`AnalysisException` when the function cannot be found.
.. versionadded:: 3.4.0
Parameters
----------
functionName : str
name of the function to check existence.
Returns
-------
:class:`Function`
The function found by the name.
Examples
--------
>>> _ = spark.sql(
... "CREATE FUNCTION my_func1 AS 'test.org.apache.spark.sql.MyDoubleAvg'")
>>> spark.catalog.getFunction("my_func1")
Function(name='my_func1', catalog='spark_catalog', namespace=['default'], ...
Using the fully qualified name for function name.
>>> spark.catalog.getFunction("default.my_func1")
Function(name='my_func1', catalog='spark_catalog', namespace=['default'], ...
>>> spark.catalog.getFunction("spark_catalog.default.my_func1")
Function(name='my_func1', catalog='spark_catalog', namespace=['default'], ...
Throw an analysis exception when the function does not exists.
>>> spark.catalog.getFunction("my_func2")
Traceback (most recent call last):
...
AnalysisException: ...
"""
jfunction = self._jcatalog.getFunction(functionName)
jnamespace = jfunction.namespace()
if jnamespace is not None:
namespace = [jnamespace[i] for i in range(0, len(jnamespace))]
else:
namespace = None
return Function(
name=jfunction.name(),
catalog=jfunction.catalog(),
namespace=namespace,
description=jfunction.description(),
className=jfunction.className(),
isTemporary=jfunction.isTemporary(),
)
def listColumns(self, tableName: str, dbName: Optional[str] = None) -> List[Column]:
"""Returns a list of columns for the given table/view in the specified database.
.. versionadded:: 2.0.0
Parameters
----------
tableName : str
name of the table to list columns.
.. versionchanged:: 3.4.0
Allow ``tableName`` to be qualified with catalog name when ``dbName`` is None.
dbName : str, optional
name of the database to find the table to list columns.
Returns
-------
list
A list of :class:`Column`.
Notes
-----
The order of arguments here is different from that of its JVM counterpart
because Python does not support method overloading.
If no database is specified, the current database and catalog
are used. This API includes all temporary views.
Examples
--------
>>> _ = spark.sql("DROP TABLE IF EXISTS tbl1")
>>> _ = spark.sql("CREATE TABLE tblA (name STRING, age INT) USING parquet")
>>> spark.catalog.listColumns("tblA")
[Column(name='name', description=None, dataType='string', nullable=True, ...
>>> _ = spark.sql("DROP TABLE tblA")
"""
if dbName is None:
iter = self._jcatalog.listColumns(tableName).toLocalIterator()
else:
warnings.warn(
"`dbName` has been deprecated since Spark 3.4 and might be removed in "
"a future version. Use listColumns(`dbName.tableName`) instead.",
FutureWarning,
)
iter = self._jcatalog.listColumns(dbName, tableName).toLocalIterator()
columns = []
while iter.hasNext():
jcolumn = iter.next()
columns.append(
Column(
name=jcolumn.name(),
description=jcolumn.description(),
dataType=jcolumn.dataType(),
nullable=jcolumn.nullable(),
isPartition=jcolumn.isPartition(),
isBucket=jcolumn.isBucket(),
)
)
return columns
def tableExists(self, tableName: str, dbName: Optional[str] = None) -> bool:
"""Check if the table or view with the specified name exists.
This can either be a temporary view or a table/view.
.. versionadded:: 3.3.0
Parameters
----------
tableName : str
name of the table to check existence.
If no database is specified, first try to treat ``tableName`` as a
multi-layer-namespace identifier, then try ``tableName`` as a normal table
name in the current database if necessary.
.. versionchanged:: 3.4.0
Allow ``tableName`` to be qualified with catalog name when ``dbName`` is None.
dbName : str, optional
name of the database to check table existence in.
Returns
-------
bool
Indicating whether the table/view exists
Examples
--------
This function can check if a table is defined or not:
>>> spark.catalog.tableExists("unexisting_table")
False
>>> _ = spark.sql("DROP TABLE IF EXISTS tbl1")
>>> _ = spark.sql("CREATE TABLE tbl1 (name STRING, age INT) USING parquet")
>>> spark.catalog.tableExists("tbl1")
True
Using the fully qualified names for tables.
>>> spark.catalog.tableExists("default.tbl1")
True
>>> spark.catalog.tableExists("spark_catalog.default.tbl1")
True
>>> spark.catalog.tableExists("tbl1", "default")
True
>>> _ = spark.sql("DROP TABLE tbl1")
Check if views exist:
>>> spark.catalog.tableExists("view1")
False
>>> _ = spark.sql("CREATE VIEW view1 AS SELECT 1")
>>> spark.catalog.tableExists("view1")
True
Using the fully qualified names for views.
>>> spark.catalog.tableExists("default.view1")
True
>>> spark.catalog.tableExists("spark_catalog.default.view1")
True
>>> spark.catalog.tableExists("view1", "default")
True
>>> _ = spark.sql("DROP VIEW view1")
Check if temporary views exist:
>>> _ = spark.sql("CREATE TEMPORARY VIEW view1 AS SELECT 1")
>>> spark.catalog.tableExists("view1")
True
>>> df = spark.sql("DROP VIEW view1")
>>> spark.catalog.tableExists("view1")
False
"""
if dbName is None:
return self._jcatalog.tableExists(tableName)
else:
warnings.warn(
"`dbName` has been deprecated since Spark 3.4 and might be removed in "
"a future version. Use tableExists(`dbName.tableName`) instead.",
FutureWarning,
)
return self._jcatalog.tableExists(dbName, tableName)
def createExternalTable(
self,
tableName: str,
path: Optional[str] = None,
source: Optional[str] = None,
schema: Optional[StructType] = None,
**options: str,
) -> DataFrame:
"""Creates a table based on the dataset in a data source.
It returns the DataFrame associated with the external table.
The data source is specified by the ``source`` and a set of ``options``.
If ``source`` is not specified, the default data source configured by
``spark.sql.sources.default`` will be used.
Optionally, a schema can be provided as the schema of the returned :class:`DataFrame` and
created external table.
.. versionadded:: 2.0.0
Returns
-------
:class:`DataFrame`
"""
warnings.warn(
"createExternalTable is deprecated since Spark 2.2, please use createTable instead.",
FutureWarning,
)
return self.createTable(tableName, path, source, schema, **options)
def createTable(
self,
tableName: str,
path: Optional[str] = None,
source: Optional[str] = None,
schema: Optional[StructType] = None,
description: Optional[str] = None,
**options: str,
) -> DataFrame:
"""Creates a table based on the dataset in a data source.
.. versionadded:: 2.2.0
Parameters
----------
tableName : str
name of the table to create.
.. versionchanged:: 3.4.0
Allow ``tableName`` to be qualified with catalog name.
path : str, optional
the path in which the data for this table exists.
When ``path`` is specified, an external table is
created from the data at the given path. Otherwise a managed table is created.
source : str, optional
the source of this table such as 'parquet, 'orc', etc.
If ``source`` is not specified, the default data source configured by
``spark.sql.sources.default`` will be used.
schema : class:`StructType`, optional
the schema for this table.
description : str, optional
the description of this table.
.. versionchanged:: 3.1.0
Added the ``description`` parameter.
**options : dict, optional
extra options to specify in the table.
Returns
-------
:class:`DataFrame`
The DataFrame associated with the table.
Examples
--------
Creating a managed table.
>>> _ = spark.catalog.createTable("tbl1", schema=spark.range(1).schema, source='parquet')
>>> _ = spark.sql("DROP TABLE tbl1")
Creating an external table
>>> import tempfile
>>> with tempfile.TemporaryDirectory() as d:
... _ = spark.catalog.createTable(
... "tbl2", schema=spark.range(1).schema, path=d, source='parquet')
>>> _ = spark.sql("DROP TABLE tbl2")
"""
if path is not None:
options["path"] = path
if source is None:
c = self._sparkSession._jconf
source = c.defaultDataSourceName()
if description is None:
description = ""
if schema is None:
df = self._jcatalog.createTable(tableName, source, description, options)
else:
if not isinstance(schema, StructType):
raise TypeError("schema should be StructType")
scala_datatype = self._jsparkSession.parseDataType(schema.json())
df = self._jcatalog.createTable(tableName, source, scala_datatype, description, options)
return DataFrame(df, self._sparkSession)
def dropTempView(self, viewName: str) -> bool:
"""Drops the local temporary view with the given view name in the catalog.
If the view has been cached before, then it will also be uncached.
Returns true if this view is dropped successfully, false otherwise.
.. versionadded:: 2.0.0
Parameters
----------
viewName : str
name of the temporary view to drop.
Returns
-------
bool
If the temporary view was successfully dropped or not.
.. versionadded:: 2.1.0
The return type of this method was ``None`` in Spark 2.0, but changed to ``bool``
in Spark 2.1.
Examples
--------
>>> spark.createDataFrame([(1, 1)]).createTempView("my_table")
Dropping the temporary view.
>>> spark.catalog.dropTempView("my_table")
True
Throw an exception if the temporary view does not exists.
>>> spark.table("my_table")
Traceback (most recent call last):
...
AnalysisException: ...
"""
return self._jcatalog.dropTempView(viewName)
def dropGlobalTempView(self, viewName: str) -> bool:
"""Drops the global temporary view with the given view name in the catalog.
.. versionadded:: 2.1.0
Parameters
----------
viewName : str
name of the global view to drop.
Returns
-------
bool
If the global view was successfully dropped or not.
Notes
-----
If the view has been cached before, then it will also be uncached.
Examples
--------
>>> spark.createDataFrame([(1, 1)]).createGlobalTempView("my_table")
Dropping the global view.
>>> spark.catalog.dropGlobalTempView("my_table")
True
Throw an exception if the global view does not exists.
>>> spark.table("global_temp.my_table")
Traceback (most recent call last):
...
AnalysisException: ...
"""
return self._jcatalog.dropGlobalTempView(viewName)
def registerFunction(
self, name: str, f: Callable[..., Any], returnType: Optional["DataType"] = None
) -> "UserDefinedFunctionLike":
"""An alias for :func:`spark.udf.register`.
See :meth:`pyspark.sql.UDFRegistration.register`.
.. versionadded:: 2.0.0
.. deprecated:: 2.3.0
Use :func:`spark.udf.register` instead.
.. versionchanged:: 3.4.0
Supports Spark Connect.
"""
warnings.warn("Deprecated in 2.3.0. Use spark.udf.register instead.", FutureWarning)
return self._sparkSession.udf.register(name, f, returnType)
def isCached(self, tableName: str) -> bool:
"""
Returns true if the table is currently cached in-memory.
.. versionadded:: 2.0.0
Parameters
----------
tableName : str
name of the table to get.
.. versionchanged:: 3.4.0
Allow ``tableName`` to be qualified with catalog name.
Returns
-------
bool
Examples
--------
>>> _ = spark.sql("DROP TABLE IF EXISTS tbl1")
>>> _ = spark.sql("CREATE TABLE tbl1 (name STRING, age INT) USING parquet")
>>> spark.catalog.cacheTable("tbl1")
>>> spark.catalog.isCached("tbl1")
True
Throw an analysis exception when the table does not exist.
>>> spark.catalog.isCached("not_existing_table")
Traceback (most recent call last):
...
AnalysisException: ...
Using the fully qualified name for the table.
>>> spark.catalog.isCached("spark_catalog.default.tbl1")
True
>>> spark.catalog.uncacheTable("tbl1")
>>> _ = spark.sql("DROP TABLE tbl1")
"""
return self._jcatalog.isCached(tableName)
def cacheTable(self, tableName: str, storageLevel: Optional[StorageLevel] = None) -> None:
"""Caches the specified table in-memory or with given storage level.
Default MEMORY_AND_DISK.