forked from Kyligence/spark
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathreadwriter.py
2149 lines (1846 loc) · 74.6 KB
/
readwriter.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
from typing import cast, overload, Dict, Iterable, List, Optional, Tuple, TYPE_CHECKING, Union
from py4j.java_gateway import JavaClass, JavaObject
from pyspark import RDD, since
from pyspark.sql.column import _to_seq, _to_java_column, Column
from pyspark.sql.types import StructType
from pyspark.sql import utils
from pyspark.sql.utils import to_str
from pyspark.errors import PySparkTypeError
if TYPE_CHECKING:
from pyspark.sql._typing import OptionalPrimitiveType, ColumnOrName
from pyspark.sql.session import SparkSession
from pyspark.sql.dataframe import DataFrame
from pyspark.sql.streaming import StreamingQuery
__all__ = ["DataFrameReader", "DataFrameWriter", "DataFrameWriterV2"]
PathOrPaths = Union[str, List[str]]
TupleOrListOfString = Union[List[str], Tuple[str, ...]]
class OptionUtils:
def _set_opts(
self,
schema: Optional[Union[StructType, str]] = None,
**options: "OptionalPrimitiveType",
) -> None:
"""
Set named options (filter out those the value is None)
"""
if schema is not None:
self.schema(schema) # type: ignore[attr-defined]
for k, v in options.items():
if v is not None:
self.option(k, v) # type: ignore[attr-defined]
class DataFrameReader(OptionUtils):
"""
Interface used to load a :class:`DataFrame` from external storage systems
(e.g. file systems, key-value stores, etc). Use :attr:`SparkSession.read`
to access this.
.. versionadded:: 1.4.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
"""
def __init__(self, spark: "SparkSession"):
self._jreader = spark._jsparkSession.read()
self._spark = spark
def _df(self, jdf: JavaObject) -> "DataFrame":
from pyspark.sql.dataframe import DataFrame
return DataFrame(jdf, self._spark)
def format(self, source: str) -> "DataFrameReader":
"""Specifies the input data source format.
.. versionadded:: 1.4.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Parameters
----------
source : str
string, name of the data source, e.g. 'json', 'parquet'.
Examples
--------
>>> spark.read.format('json')
<...readwriter.DataFrameReader object ...>
Write a DataFrame into a JSON file and read it back.
>>> import tempfile
>>> with tempfile.TemporaryDirectory() as d:
... # Write a DataFrame into a JSON file
... spark.createDataFrame(
... [{"age": 100, "name": "Hyukjin Kwon"}]
... ).write.mode("overwrite").format("json").save(d)
...
... # Read the JSON file as a DataFrame.
... spark.read.format('json').load(d).show()
+---+------------+
|age| name|
+---+------------+
|100|Hyukjin Kwon|
+---+------------+
"""
self._jreader = self._jreader.format(source)
return self
def schema(self, schema: Union[StructType, str]) -> "DataFrameReader":
"""Specifies the input schema.
Some data sources (e.g. JSON) can infer the input schema automatically from data.
By specifying the schema here, the underlying data source can skip the schema
inference step, and thus speed up data loading.
.. versionadded:: 1.4.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Parameters
----------
schema : :class:`pyspark.sql.types.StructType` or str
a :class:`pyspark.sql.types.StructType` object or a DDL-formatted string
(For example ``col0 INT, col1 DOUBLE``).
Examples
--------
>>> spark.read.schema("col0 INT, col1 DOUBLE")
<...readwriter.DataFrameReader object ...>
Specify the schema with reading a CSV file.
>>> import tempfile
>>> with tempfile.TemporaryDirectory() as d:
... spark.read.schema("col0 INT, col1 DOUBLE").format("csv").load(d).printSchema()
root
|-- col0: integer (nullable = true)
|-- col1: double (nullable = true)
"""
from pyspark.sql import SparkSession
spark = SparkSession._getActiveSessionOrCreate()
if isinstance(schema, StructType):
jschema = spark._jsparkSession.parseDataType(schema.json())
self._jreader = self._jreader.schema(jschema)
elif isinstance(schema, str):
self._jreader = self._jreader.schema(schema)
else:
raise PySparkTypeError(
error_class="NOT_STR_OR_STRUCT",
message_parameters={
"arg_name": "schema",
"arg_type": type(schema).__name__,
},
)
return self
def option(self, key: str, value: "OptionalPrimitiveType") -> "DataFrameReader":
"""
Adds an input option for the underlying data source.
.. versionadded:: 1.5.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Parameters
----------
key : str
The key for the option to set.
value
The value for the option to set.
Examples
--------
>>> spark.read.option("key", "value")
<...readwriter.DataFrameReader object ...>
Specify the option 'nullValue' with reading a CSV file.
>>> import tempfile
>>> with tempfile.TemporaryDirectory() as d:
... # Write a DataFrame into a CSV file
... df = spark.createDataFrame([{"age": 100, "name": "Hyukjin Kwon"}])
... df.write.mode("overwrite").format("csv").save(d)
...
... # Read the CSV file as a DataFrame with 'nullValue' option set to 'Hyukjin Kwon'.
... spark.read.schema(df.schema).option(
... "nullValue", "Hyukjin Kwon").format('csv').load(d).show()
+---+----+
|age|name|
+---+----+
|100|NULL|
+---+----+
"""
self._jreader = self._jreader.option(key, to_str(value))
return self
def options(self, **options: "OptionalPrimitiveType") -> "DataFrameReader":
"""
Adds input options for the underlying data source.
.. versionadded:: 1.4.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Parameters
----------
**options : dict
The dictionary of string keys and prmitive-type values.
Examples
--------
>>> spark.read.option("key", "value")
<...readwriter.DataFrameReader object ...>
Specify the option 'nullValue' and 'header' with reading a CSV file.
>>> import tempfile
>>> with tempfile.TemporaryDirectory() as d:
... # Write a DataFrame into a CSV file with a header.
... df = spark.createDataFrame([{"age": 100, "name": "Hyukjin Kwon"}])
... df.write.option("header", True).mode("overwrite").format("csv").save(d)
...
... # Read the CSV file as a DataFrame with 'nullValue' option set to 'Hyukjin Kwon',
... # and 'header' option set to `True`.
... spark.read.options(
... nullValue="Hyukjin Kwon",
... header=True
... ).format('csv').load(d).show()
+---+----+
|age|name|
+---+----+
|100|NULL|
+---+----+
"""
for k in options:
self._jreader = self._jreader.option(k, to_str(options[k]))
return self
def load(
self,
path: Optional[PathOrPaths] = None,
format: Optional[str] = None,
schema: Optional[Union[StructType, str]] = None,
**options: "OptionalPrimitiveType",
) -> "DataFrame":
"""Loads data from a data source and returns it as a :class:`DataFrame`.
.. versionadded:: 1.4.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Parameters
----------
path : str or list, optional
optional string or a list of string for file-system backed data sources.
format : str, optional
optional string for format of the data source. Default to 'parquet'.
schema : :class:`pyspark.sql.types.StructType` or str, optional
optional :class:`pyspark.sql.types.StructType` for the input schema
or a DDL-formatted string (For example ``col0 INT, col1 DOUBLE``).
**options : dict
all other string options
Examples
--------
Load a CSV file with format, schema and options specified.
>>> import tempfile
>>> with tempfile.TemporaryDirectory() as d:
... # Write a DataFrame into a CSV file with a header
... df = spark.createDataFrame([{"age": 100, "name": "Hyukjin Kwon"}])
... df.write.option("header", True).mode("overwrite").format("csv").save(d)
...
... # Read the CSV file as a DataFrame with 'nullValue' option set to 'Hyukjin Kwon',
... # and 'header' option set to `True`.
... df = spark.read.load(
... d, schema=df.schema, format="csv", nullValue="Hyukjin Kwon", header=True)
... df.printSchema()
... df.show()
root
|-- age: long (nullable = true)
|-- name: string (nullable = true)
+---+----+
|age|name|
+---+----+
|100|NULL|
+---+----+
"""
if format is not None:
self.format(format)
if schema is not None:
self.schema(schema)
self.options(**options)
if isinstance(path, str):
return self._df(self._jreader.load(path))
elif path is not None:
if type(path) != list:
path = [path] # type: ignore[list-item]
assert self._spark._sc._jvm is not None
return self._df(self._jreader.load(self._spark._sc._jvm.PythonUtils.toSeq(path)))
else:
return self._df(self._jreader.load())
def json(
self,
path: Union[str, List[str], RDD[str]],
schema: Optional[Union[StructType, str]] = None,
primitivesAsString: Optional[Union[bool, str]] = None,
prefersDecimal: Optional[Union[bool, str]] = None,
allowComments: Optional[Union[bool, str]] = None,
allowUnquotedFieldNames: Optional[Union[bool, str]] = None,
allowSingleQuotes: Optional[Union[bool, str]] = None,
allowNumericLeadingZero: Optional[Union[bool, str]] = None,
allowBackslashEscapingAnyCharacter: Optional[Union[bool, str]] = None,
mode: Optional[str] = None,
columnNameOfCorruptRecord: Optional[str] = None,
dateFormat: Optional[str] = None,
timestampFormat: Optional[str] = None,
multiLine: Optional[Union[bool, str]] = None,
allowUnquotedControlChars: Optional[Union[bool, str]] = None,
lineSep: Optional[str] = None,
samplingRatio: Optional[Union[float, str]] = None,
dropFieldIfAllNull: Optional[Union[bool, str]] = None,
encoding: Optional[str] = None,
locale: Optional[str] = None,
pathGlobFilter: Optional[Union[bool, str]] = None,
recursiveFileLookup: Optional[Union[bool, str]] = None,
modifiedBefore: Optional[Union[bool, str]] = None,
modifiedAfter: Optional[Union[bool, str]] = None,
allowNonNumericNumbers: Optional[Union[bool, str]] = None,
) -> "DataFrame":
"""
Loads JSON files and returns the results as a :class:`DataFrame`.
`JSON Lines <http://jsonlines.org/>`_ (newline-delimited JSON) is supported by default.
For JSON (one record per file), set the ``multiLine`` parameter to ``true``.
If the ``schema`` parameter is not specified, this function goes
through the input once to determine the input schema.
.. versionadded:: 1.4.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Parameters
----------
path : str, list or :class:`RDD`
string represents path to the JSON dataset, or a list of paths,
or RDD of Strings storing JSON objects.
schema : :class:`pyspark.sql.types.StructType` or str, optional
an optional :class:`pyspark.sql.types.StructType` for the input schema or
a DDL-formatted string (For example ``col0 INT, col1 DOUBLE``).
Other Parameters
----------------
Extra options
For the extra options, refer to
`Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-json.html#data-source-option>`_
for the version you use.
.. # noqa
Examples
--------
Write a DataFrame into a JSON file and read it back.
>>> import tempfile
>>> with tempfile.TemporaryDirectory() as d:
... # Write a DataFrame into a JSON file
... spark.createDataFrame(
... [{"age": 100, "name": "Hyukjin Kwon"}]
... ).write.mode("overwrite").format("json").save(d)
...
... # Read the JSON file as a DataFrame.
... spark.read.json(d).show()
+---+------------+
|age| name|
+---+------------+
|100|Hyukjin Kwon|
+---+------------+
"""
self._set_opts(
schema=schema,
primitivesAsString=primitivesAsString,
prefersDecimal=prefersDecimal,
allowComments=allowComments,
allowUnquotedFieldNames=allowUnquotedFieldNames,
allowSingleQuotes=allowSingleQuotes,
allowNumericLeadingZero=allowNumericLeadingZero,
allowBackslashEscapingAnyCharacter=allowBackslashEscapingAnyCharacter,
mode=mode,
columnNameOfCorruptRecord=columnNameOfCorruptRecord,
dateFormat=dateFormat,
timestampFormat=timestampFormat,
multiLine=multiLine,
allowUnquotedControlChars=allowUnquotedControlChars,
lineSep=lineSep,
samplingRatio=samplingRatio,
dropFieldIfAllNull=dropFieldIfAllNull,
encoding=encoding,
locale=locale,
pathGlobFilter=pathGlobFilter,
recursiveFileLookup=recursiveFileLookup,
modifiedBefore=modifiedBefore,
modifiedAfter=modifiedAfter,
allowNonNumericNumbers=allowNonNumericNumbers,
)
if isinstance(path, str):
path = [path]
if type(path) == list:
assert self._spark._sc._jvm is not None
return self._df(self._jreader.json(self._spark._sc._jvm.PythonUtils.toSeq(path)))
elif isinstance(path, RDD):
def func(iterator: Iterable) -> Iterable:
for x in iterator:
if not isinstance(x, str):
x = str(x)
if isinstance(x, str):
x = x.encode("utf-8")
yield x
keyed = path.mapPartitions(func)
keyed._bypass_serializer = True # type: ignore[attr-defined]
assert self._spark._jvm is not None
jrdd = keyed._jrdd.map(self._spark._jvm.BytesToString())
return self._df(self._jreader.json(jrdd))
else:
raise PySparkTypeError(
error_class="NOT_STR_OR_LIST_OF_RDD",
message_parameters={
"arg_name": "path",
"arg_type": type(path).__name__,
},
)
def table(self, tableName: str) -> "DataFrame":
"""Returns the specified table as a :class:`DataFrame`.
.. versionadded:: 1.4.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Parameters
----------
tableName : str
string, name of the table.
Examples
--------
>>> df = spark.range(10)
>>> df.createOrReplaceTempView('tblA')
>>> spark.read.table('tblA').show()
+---+
| id|
+---+
| 0|
| 1|
| 2|
| 3|
| 4|
| 5|
| 6|
| 7|
| 8|
| 9|
+---+
>>> _ = spark.sql("DROP TABLE tblA")
"""
return self._df(self._jreader.table(tableName))
def parquet(self, *paths: str, **options: "OptionalPrimitiveType") -> "DataFrame":
"""
Loads Parquet files, returning the result as a :class:`DataFrame`.
.. versionadded:: 1.4.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Parameters
----------
paths : str
Other Parameters
----------------
**options
For the extra options, refer to
`Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#data-source-option>`_
for the version you use.
.. # noqa
Examples
--------
Write a DataFrame into a Parquet file and read it back.
>>> import tempfile
>>> with tempfile.TemporaryDirectory() as d:
... # Write a DataFrame into a Parquet file
... spark.createDataFrame(
... [{"age": 100, "name": "Hyukjin Kwon"}]
... ).write.mode("overwrite").format("parquet").save(d)
...
... # Read the Parquet file as a DataFrame.
... spark.read.parquet(d).show()
+---+------------+
|age| name|
+---+------------+
|100|Hyukjin Kwon|
+---+------------+
"""
mergeSchema = options.get("mergeSchema", None)
pathGlobFilter = options.get("pathGlobFilter", None)
modifiedBefore = options.get("modifiedBefore", None)
modifiedAfter = options.get("modifiedAfter", None)
recursiveFileLookup = options.get("recursiveFileLookup", None)
datetimeRebaseMode = options.get("datetimeRebaseMode", None)
int96RebaseMode = options.get("int96RebaseMode", None)
self._set_opts(
mergeSchema=mergeSchema,
pathGlobFilter=pathGlobFilter,
recursiveFileLookup=recursiveFileLookup,
modifiedBefore=modifiedBefore,
modifiedAfter=modifiedAfter,
datetimeRebaseMode=datetimeRebaseMode,
int96RebaseMode=int96RebaseMode,
)
return self._df(self._jreader.parquet(_to_seq(self._spark._sc, paths)))
def text(
self,
paths: PathOrPaths,
wholetext: bool = False,
lineSep: Optional[str] = None,
pathGlobFilter: Optional[Union[bool, str]] = None,
recursiveFileLookup: Optional[Union[bool, str]] = None,
modifiedBefore: Optional[Union[bool, str]] = None,
modifiedAfter: Optional[Union[bool, str]] = None,
) -> "DataFrame":
"""
Loads text files and returns a :class:`DataFrame` whose schema starts with a
string column named "value", and followed by partitioned columns if there
are any.
The text files must be encoded as UTF-8.
By default, each line in the text file is a new row in the resulting DataFrame.
.. versionadded:: 1.6.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Parameters
----------
paths : str or list
string, or list of strings, for input path(s).
Other Parameters
----------------
Extra options
For the extra options, refer to
`Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-text.html#data-source-option>`_
for the version you use.
.. # noqa
Examples
--------
Write a DataFrame into a text file and read it back.
>>> import tempfile
>>> with tempfile.TemporaryDirectory() as d:
... # Write a DataFrame into a text file
... df = spark.createDataFrame([("a",), ("b",), ("c",)], schema=["alphabets"])
... df.write.mode("overwrite").format("text").save(d)
...
... # Read the text file as a DataFrame.
... spark.read.schema(df.schema).text(d).sort("alphabets").show()
+---------+
|alphabets|
+---------+
| a|
| b|
| c|
+---------+
"""
self._set_opts(
wholetext=wholetext,
lineSep=lineSep,
pathGlobFilter=pathGlobFilter,
recursiveFileLookup=recursiveFileLookup,
modifiedBefore=modifiedBefore,
modifiedAfter=modifiedAfter,
)
if isinstance(paths, str):
paths = [paths]
assert self._spark._sc._jvm is not None
return self._df(self._jreader.text(self._spark._sc._jvm.PythonUtils.toSeq(paths)))
def csv(
self,
path: PathOrPaths,
schema: Optional[Union[StructType, str]] = None,
sep: Optional[str] = None,
encoding: Optional[str] = None,
quote: Optional[str] = None,
escape: Optional[str] = None,
comment: Optional[str] = None,
header: Optional[Union[bool, str]] = None,
inferSchema: Optional[Union[bool, str]] = None,
ignoreLeadingWhiteSpace: Optional[Union[bool, str]] = None,
ignoreTrailingWhiteSpace: Optional[Union[bool, str]] = None,
nullValue: Optional[str] = None,
nanValue: Optional[str] = None,
positiveInf: Optional[str] = None,
negativeInf: Optional[str] = None,
dateFormat: Optional[str] = None,
timestampFormat: Optional[str] = None,
maxColumns: Optional[Union[int, str]] = None,
maxCharsPerColumn: Optional[Union[int, str]] = None,
maxMalformedLogPerPartition: Optional[Union[int, str]] = None,
mode: Optional[str] = None,
columnNameOfCorruptRecord: Optional[str] = None,
multiLine: Optional[Union[bool, str]] = None,
charToEscapeQuoteEscaping: Optional[str] = None,
samplingRatio: Optional[Union[float, str]] = None,
enforceSchema: Optional[Union[bool, str]] = None,
emptyValue: Optional[str] = None,
locale: Optional[str] = None,
lineSep: Optional[str] = None,
pathGlobFilter: Optional[Union[bool, str]] = None,
recursiveFileLookup: Optional[Union[bool, str]] = None,
modifiedBefore: Optional[Union[bool, str]] = None,
modifiedAfter: Optional[Union[bool, str]] = None,
unescapedQuoteHandling: Optional[str] = None,
) -> "DataFrame":
r"""Loads a CSV file and returns the result as a :class:`DataFrame`.
This function will go through the input once to determine the input schema if
``inferSchema`` is enabled. To avoid going through the entire data once, disable
``inferSchema`` option or specify the schema explicitly using ``schema``.
.. versionadded:: 2.0.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Parameters
----------
path : str or list
string, or list of strings, for input path(s),
or RDD of Strings storing CSV rows.
schema : :class:`pyspark.sql.types.StructType` or str, optional
an optional :class:`pyspark.sql.types.StructType` for the input schema
or a DDL-formatted string (For example ``col0 INT, col1 DOUBLE``).
Other Parameters
----------------
Extra options
For the extra options, refer to
`Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-csv.html#data-source-option>`_
for the version you use.
.. # noqa
Examples
--------
Write a DataFrame into a CSV file and read it back.
>>> import tempfile
>>> with tempfile.TemporaryDirectory() as d:
... # Write a DataFrame into a CSV file
... df = spark.createDataFrame([{"age": 100, "name": "Hyukjin Kwon"}])
... df.write.mode("overwrite").format("csv").save(d)
...
... # Read the CSV file as a DataFrame with 'nullValue' option set to 'Hyukjin Kwon'.
... spark.read.csv(d, schema=df.schema, nullValue="Hyukjin Kwon").show()
+---+----+
|age|name|
+---+----+
|100|NULL|
+---+----+
"""
self._set_opts(
schema=schema,
sep=sep,
encoding=encoding,
quote=quote,
escape=escape,
comment=comment,
header=header,
inferSchema=inferSchema,
ignoreLeadingWhiteSpace=ignoreLeadingWhiteSpace,
ignoreTrailingWhiteSpace=ignoreTrailingWhiteSpace,
nullValue=nullValue,
nanValue=nanValue,
positiveInf=positiveInf,
negativeInf=negativeInf,
dateFormat=dateFormat,
timestampFormat=timestampFormat,
maxColumns=maxColumns,
maxCharsPerColumn=maxCharsPerColumn,
maxMalformedLogPerPartition=maxMalformedLogPerPartition,
mode=mode,
columnNameOfCorruptRecord=columnNameOfCorruptRecord,
multiLine=multiLine,
charToEscapeQuoteEscaping=charToEscapeQuoteEscaping,
samplingRatio=samplingRatio,
enforceSchema=enforceSchema,
emptyValue=emptyValue,
locale=locale,
lineSep=lineSep,
pathGlobFilter=pathGlobFilter,
recursiveFileLookup=recursiveFileLookup,
modifiedBefore=modifiedBefore,
modifiedAfter=modifiedAfter,
unescapedQuoteHandling=unescapedQuoteHandling,
)
if isinstance(path, str):
path = [path]
if type(path) == list:
assert self._spark._sc._jvm is not None
return self._df(self._jreader.csv(self._spark._sc._jvm.PythonUtils.toSeq(path)))
elif isinstance(path, RDD):
def func(iterator):
for x in iterator:
if not isinstance(x, str):
x = str(x)
if isinstance(x, str):
x = x.encode("utf-8")
yield x
keyed = path.mapPartitions(func)
keyed._bypass_serializer = True
jrdd = keyed._jrdd.map(self._spark._jvm.BytesToString())
# see SPARK-22112
# There aren't any jvm api for creating a dataframe from rdd storing csv.
# We can do it through creating a jvm dataset firstly and using the jvm api
# for creating a dataframe from dataset storing csv.
jdataset = self._spark._jsparkSession.createDataset(
jrdd.rdd(), self._spark._jvm.Encoders.STRING()
)
return self._df(self._jreader.csv(jdataset))
else:
raise PySparkTypeError(
error_class="NOT_STR_OR_LIST_OF_RDD",
message_parameters={
"arg_name": "path",
"arg_type": type(path).__name__,
},
)
def orc(
self,
path: PathOrPaths,
mergeSchema: Optional[bool] = None,
pathGlobFilter: Optional[Union[bool, str]] = None,
recursiveFileLookup: Optional[Union[bool, str]] = None,
modifiedBefore: Optional[Union[bool, str]] = None,
modifiedAfter: Optional[Union[bool, str]] = None,
) -> "DataFrame":
"""Loads ORC files, returning the result as a :class:`DataFrame`.
.. versionadded:: 1.5.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Parameters
----------
path : str or list
Other Parameters
----------------
Extra options
For the extra options, refer to
`Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-orc.html#data-source-option>`_
for the version you use.
.. # noqa
Examples
--------
Write a DataFrame into a ORC file and read it back.
>>> import tempfile
>>> with tempfile.TemporaryDirectory() as d:
... # Write a DataFrame into a ORC file
... spark.createDataFrame(
... [{"age": 100, "name": "Hyukjin Kwon"}]
... ).write.mode("overwrite").format("orc").save(d)
...
... # Read the Parquet file as a DataFrame.
... spark.read.orc(d).show()
+---+------------+
|age| name|
+---+------------+
|100|Hyukjin Kwon|
+---+------------+
"""
self._set_opts(
mergeSchema=mergeSchema,
pathGlobFilter=pathGlobFilter,
modifiedBefore=modifiedBefore,
modifiedAfter=modifiedAfter,
recursiveFileLookup=recursiveFileLookup,
)
if isinstance(path, str):
path = [path]
return self._df(self._jreader.orc(_to_seq(self._spark._sc, path)))
@overload
def jdbc(
self, url: str, table: str, *, properties: Optional[Dict[str, str]] = None
) -> "DataFrame":
...
@overload
def jdbc(
self,
url: str,
table: str,
column: str,
lowerBound: Union[int, str],
upperBound: Union[int, str],
numPartitions: int,
*,
properties: Optional[Dict[str, str]] = None,
) -> "DataFrame":
...
@overload
def jdbc(
self,
url: str,
table: str,
*,
predicates: List[str],
properties: Optional[Dict[str, str]] = None,
) -> "DataFrame":
...
def jdbc(
self,
url: str,
table: str,
column: Optional[str] = None,
lowerBound: Optional[Union[int, str]] = None,
upperBound: Optional[Union[int, str]] = None,
numPartitions: Optional[int] = None,
predicates: Optional[List[str]] = None,
properties: Optional[Dict[str, str]] = None,
) -> "DataFrame":
"""
Construct a :class:`DataFrame` representing the database table named ``table``
accessible via JDBC URL ``url`` and connection ``properties``.
Partitions of the table will be retrieved in parallel if either ``column`` or
``predicates`` is specified. ``lowerBound``, ``upperBound`` and ``numPartitions``
is needed when ``column`` is specified.
If both ``column`` and ``predicates`` are specified, ``column`` will be used.
.. versionadded:: 1.4.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Parameters
----------
table : str
the name of the table
column : str, optional
alias of ``partitionColumn`` option. Refer to ``partitionColumn`` in
`Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-jdbc.html#data-source-option>`_
for the version you use.
predicates : list, optional
a list of expressions suitable for inclusion in WHERE clauses;
each one defines one partition of the :class:`DataFrame`
properties : dict, optional
a dictionary of JDBC database connection arguments. Normally at
least properties "user" and "password" with their corresponding values.
For example { 'user' : 'SYSTEM', 'password' : 'mypassword' }
Other Parameters
----------------
Extra options
For the extra options, refer to
`Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-jdbc.html#data-source-option>`_
for the version you use.
.. # noqa
Notes
-----
Don't create too many partitions in parallel on a large cluster;
otherwise Spark might crash your external database systems.
Returns
-------
:class:`DataFrame`
"""
if properties is None:
properties = dict()
assert self._spark._sc._gateway is not None
jprop = JavaClass(
"java.util.Properties",
self._spark._sc._gateway._gateway_client,
)()
for k in properties:
jprop.setProperty(k, properties[k])
if column is not None:
assert lowerBound is not None, "lowerBound can not be None when ``column`` is specified"
assert upperBound is not None, "upperBound can not be None when ``column`` is specified"
assert (
numPartitions is not None
), "numPartitions can not be None when ``column`` is specified"
return self._df(
self._jreader.jdbc(
url, table, column, int(lowerBound), int(upperBound), int(numPartitions), jprop
)
)
if predicates is not None:
gateway = self._spark._sc._gateway
assert gateway is not None
jpredicates = utils.toJArray(gateway, gateway.jvm.java.lang.String, predicates)
return self._df(self._jreader.jdbc(url, table, jpredicates, jprop))
return self._df(self._jreader.jdbc(url, table, jprop))
class DataFrameWriter(OptionUtils):
"""
Interface used to write a :class:`DataFrame` to external storage systems
(e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
to access this.
.. versionadded:: 1.4.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
"""
def __init__(self, df: "DataFrame"):
self._df = df
self._spark = df.sparkSession
self._jwrite = df._jdf.write()
def _sq(self, jsq: JavaObject) -> "StreamingQuery":
from pyspark.sql.streaming import StreamingQuery
return StreamingQuery(jsq)
def mode(self, saveMode: Optional[str]) -> "DataFrameWriter":
"""Specifies the behavior when data or table already exists.
Options include:
* `append`: Append contents of this :class:`DataFrame` to existing data.
* `overwrite`: Overwrite existing data.
* `error` or `errorifexists`: Throw an exception if data already exists.
* `ignore`: Silently ignore this operation if data already exists.
.. versionadded:: 1.4.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Examples
--------
Raise an error when writing to an existing path.
>>> import tempfile
>>> with tempfile.TemporaryDirectory() as d:
... spark.createDataFrame(
... [{"age": 80, "name": "Xinrong Meng"}]
... ).write.mode("error").format("parquet").save(d) # doctest: +SKIP
Traceback (most recent call last):
...
...AnalysisException: ...
Write a Parquet file back with various options, and read it back.