forked from apache/spark
-
Notifications
You must be signed in to change notification settings - Fork 5
/
Copy pathnamespace.py
3902 lines (3343 loc) · 134 KB
/
namespace.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.
#
"""
Wrappers around spark that correspond to common pandas functions.
"""
from typing import (
Any,
Callable,
Dict,
List,
Optional,
Set,
Sized,
Tuple,
Type,
Union,
cast,
no_type_check,
)
from collections.abc import Iterable
from datetime import tzinfo
from functools import reduce
from io import BytesIO
import json
import warnings
import numpy as np
import pandas as pd
from pandas.api.types import ( # type: ignore[attr-defined]
is_datetime64_dtype,
is_list_like,
)
from pandas.tseries.offsets import DateOffset
import pyarrow as pa
import pyarrow.parquet as pq
from pyspark.sql import functions as F, Column as PySparkColumn
from pyspark.sql.functions import pandas_udf
from pyspark.sql.types import (
ByteType,
ShortType,
IntegerType,
LongType,
FloatType,
DoubleType,
BooleanType,
TimestampType,
TimestampNTZType,
DecimalType,
StringType,
DateType,
StructType,
StructField,
DataType,
)
from pyspark.sql.dataframe import DataFrame as PySparkDataFrame
from pyspark import pandas as ps
from pyspark.pandas._typing import Axis, Dtype, Label, Name
from pyspark.pandas.base import IndexOpsMixin
from pyspark.pandas.utils import (
align_diff_frames,
default_session,
is_name_like_tuple,
is_name_like_value,
name_like_string,
same_anchor,
scol_for,
validate_axis,
log_advice,
)
from pyspark.pandas.frame import DataFrame, _reduce_spark_multi
from pyspark.pandas.internal import (
InternalFrame,
InternalField,
DEFAULT_SERIES_NAME,
HIDDEN_COLUMNS,
SPARK_INDEX_NAME_FORMAT,
NATURAL_ORDER_COLUMN_NAME,
)
from pyspark.pandas.series import Series, first_series
from pyspark.pandas.spark.utils import as_nullable_spark_type, force_decimal_precision_scale
from pyspark.pandas.indexes import Index, DatetimeIndex, TimedeltaIndex
from pyspark.pandas.indexes.multi import MultiIndex
__all__ = [
"from_pandas",
"range",
"read_csv",
"read_delta",
"read_table",
"read_spark_io",
"read_parquet",
"read_clipboard",
"read_excel",
"read_html",
"to_datetime",
"date_range",
"to_timedelta",
"timedelta_range",
"get_dummies",
"concat",
"melt",
"isna",
"isnull",
"notna",
"notnull",
"read_sql_table",
"read_sql_query",
"read_sql",
"read_json",
"merge",
"merge_asof",
"to_numeric",
"broadcast",
"read_orc",
"json_normalize",
]
def from_pandas(pobj: Union[pd.DataFrame, pd.Series, pd.Index]) -> Union[Series, DataFrame, Index]:
"""Create a pandas-on-Spark DataFrame, Series or Index from a pandas DataFrame, Series or Index.
This is similar to Spark's `SparkSession.createDataFrame()` with pandas DataFrame,
but this also works with pandas Series and picks the index.
Parameters
----------
pobj : pandas.DataFrame, pandas.Series or pandas.Index
pandas DataFrame, Series or Index to read.
Returns
-------
DataFrame, Series or Index
If a pandas DataFrame is passed in, this function returns a pandas-on-Spark DataFrame.
If a pandas Series is passed in, this function returns a pandas-on-Spark Series.
If a pandas Index is passed in, this function returns a pandas-on-Spark Index.
Examples
--------
>>> import pandas as pd
>>> import pyspark.pandas as ps
Convert a pandas DataFrame:
>>> pdf = pd.DataFrame({'a': [1, 2, 3]})
>>> psdf = ps.from_pandas(pdf)
>>> psdf
a
0 1
1 2
2 3
Convert a pandas Series:
>>> pser = pd.Series([1, 2, 3])
>>> psser = ps.from_pandas(pser)
>>> psser
0 1
1 2
2 3
dtype: int64
Convert a pandas Index:
>>> pidx = pd.Index([1, 2, 3])
>>> psidx = ps.from_pandas(pidx)
>>> psidx
Index([1, 2, 3], dtype='int64')
"""
if isinstance(pobj, pd.Series):
return Series(pobj)
elif isinstance(pobj, pd.DataFrame):
return DataFrame(pobj)
elif isinstance(pobj, pd.Index):
return DataFrame(pd.DataFrame(index=pobj)).index
else:
raise TypeError("Unknown data type: {}".format(type(pobj).__name__))
# built-in range
_range: Type[range] = range # type: ignore[assignment]
def range(
start: int, end: Optional[int] = None, step: int = 1, num_partitions: Optional[int] = None
) -> DataFrame:
"""
Create a DataFrame with some range of numbers.
The resulting DataFrame has a single int64 column named `id`, containing elements in a range
from ``start`` to ``end`` (exclusive) with step value ``step``. If only the first parameter
(i.e. start) is specified, we treat it as the end value with the start value being 0.
This is like the range function in SparkSession and is used primarily for testing.
Parameters
----------
start : int
the start value (inclusive)
end : int, optional
the end value (exclusive)
step : int, optional, default 1
the incremental step
num_partitions : int, optional
the number of partitions of the DataFrame
Returns
-------
DataFrame
Examples
--------
When the first parameter is specified, we generate a range of values up till that number.
>>> ps.range(5)
id
0 0
1 1
2 2
3 3
4 4
When start, end, and step are specified:
>>> ps.range(start = 100, end = 200, step = 20)
id
0 100
1 120
2 140
3 160
4 180
"""
sdf = default_session().range(start=start, end=end, step=step, numPartitions=num_partitions)
return DataFrame(sdf)
def read_csv(
path: Union[str, List[str]],
sep: str = ",",
header: Union[str, int, None] = "infer",
names: Optional[Union[str, List[str]]] = None,
index_col: Optional[Union[str, List[str]]] = None,
usecols: Optional[Union[List[int], List[str], Callable[[str], bool]]] = None,
dtype: Optional[Union[str, Dtype, Dict[str, Union[str, Dtype]]]] = None,
nrows: Optional[int] = None,
parse_dates: bool = False,
quotechar: Optional[str] = None,
escapechar: Optional[str] = None,
comment: Optional[str] = None,
encoding: Optional[str] = None,
**options: Any,
) -> Union[DataFrame, Series]:
"""Read CSV (comma-separated) file into DataFrame or Series.
Parameters
----------
path : str or list
Path(s) of the CSV file(s) to be read.
sep : str, default ‘,’
Delimiter to use. Non empty string.
header : int, default ‘infer’
Whether to use the column names, and the start of the data.
Default behavior is to infer the column names: if no names are passed
the behavior is identical to `header=0` and column names are inferred from
the first line of the file, if column names are passed explicitly then
the behavior is identical to `header=None`. Explicitly pass `header=0` to be
able to replace existing names
names : str or array-like, optional
List of column names to use. If file contains no header row, then you should
explicitly pass `header=None`. Duplicates in this list will cause an error to be issued.
If a string is given, it should be a DDL-formatted string in Spark SQL, which is
preferred to avoid schema inference for better performance.
index_col: str or list of str, optional, default: None
Index column of table in Spark.
usecols : list-like or callable, optional
Return a subset of the columns. If list-like, all elements must either be
positional (i.e. integer indices into the document columns) or strings that
correspond to column names provided either by the user in names or inferred
from the document header row(s).
If callable, the callable function will be evaluated against the column names,
returning names where the callable function evaluates to `True`.
dtype : Type name or dict of column -> type, default None
Data type for data or columns. E.g. {‘a’: np.float64, ‘b’: np.int32} Use str or object
together with suitable na_values settings to preserve and not interpret dtype.
nrows : int, default None
Number of rows to read from the CSV file.
parse_dates : boolean or list of ints or names or list of lists or dict, default `False`.
Currently only `False` is allowed.
quotechar : str (length 1), optional
The character used to denote the start and end of a quoted item. Quoted items can include
the delimiter and it will be ignored.
escapechar : str (length 1), default None
One-character string used to escape other characters.
comment: str, optional
Indicates the line should not be parsed.
encoding: str, optional
Indicates the encoding to read file
options : dict
All other options passed directly into Spark's data source.
Returns
-------
DataFrame or Series
See Also
--------
DataFrame.to_csv : Write DataFrame to a comma-separated values (csv) file.
Examples
--------
>>> ps.read_csv('data.csv') # doctest: +SKIP
Load multiple CSV files as a single DataFrame:
>>> ps.read_csv(['data-01.csv', 'data-02.csv']) # doctest: +SKIP
"""
# For latin-1 encoding is same as iso-8859-1, that's why its mapped to iso-8859-1.
encoding_mapping = {"latin-1": "iso-8859-1"}
if "options" in options and isinstance(options.get("options"), dict) and len(options) == 1:
options = options.get("options")
if parse_dates is not False:
raise ValueError("parse_dates can only be `False`: %s" % parse_dates)
if usecols is not None and not callable(usecols):
usecols = list(usecols) # type: ignore[assignment]
if usecols is None or callable(usecols) or len(usecols) > 0:
reader = default_session().read
reader.option("inferSchema", True)
reader.option("sep", sep)
if header == "infer":
header = 0 if names is None else None
if header == 0:
reader.option("header", True)
elif header is None:
reader.option("header", False)
else:
raise ValueError("Unknown header argument {}".format(header))
if quotechar is not None:
reader.option("quote", quotechar)
if escapechar is not None:
reader.option("escape", escapechar)
if comment is not None:
if not isinstance(comment, str) or len(comment) != 1:
raise ValueError("Only length-1 comment characters supported")
reader.option("comment", comment)
reader.options(**options)
if encoding is not None:
reader.option("encoding", encoding_mapping.get(encoding, encoding))
column_labels: Dict[Any, str]
if isinstance(names, str):
sdf = reader.schema(names).csv(path)
column_labels = {col: col for col in sdf.columns}
else:
sdf = reader.csv(path)
if is_list_like(names):
names = list(names)
if len(set(names)) != len(names):
raise ValueError("Found non-unique column index")
if len(names) != len(sdf.columns):
raise ValueError(
"The number of names [%s] does not match the number "
"of columns [%d]. Try names by a Spark SQL DDL-formatted "
"string." % (len(sdf.schema), len(names))
)
column_labels = dict(zip(names, sdf.columns))
elif header is None:
column_labels = dict(enumerate(sdf.columns))
else:
column_labels = {col: col for col in sdf.columns}
if usecols is not None:
missing: List[Union[int, str]]
if callable(usecols):
column_labels = {
label: col for label, col in column_labels.items() if usecols(label)
}
missing = []
elif all(isinstance(col, int) for col in usecols):
usecols_ints = cast(List[int], usecols)
new_column_labels = {
label: col
for i, (label, col) in enumerate(column_labels.items())
if i in usecols_ints
}
missing = [
col
for col in usecols_ints
if (
col >= len(column_labels)
or list(column_labels)[col] not in new_column_labels
)
]
column_labels = new_column_labels
elif all(isinstance(col, str) for col in usecols):
new_column_labels = {
label: col for label, col in column_labels.items() if label in usecols
}
missing = [col for col in usecols if col not in new_column_labels]
column_labels = new_column_labels
else:
raise ValueError(
"'usecols' must either be list-like of all strings, "
"all unicode, all integers or a callable."
)
if len(missing) > 0:
raise ValueError(
"Usecols do not match columns, columns expected but not " "found: %s" % missing
)
if len(column_labels) > 0:
sdf = sdf.select([scol_for(sdf, col) for col in column_labels.values()])
else:
sdf = default_session().createDataFrame([], schema=StructType())
else:
sdf = default_session().createDataFrame([], schema=StructType())
column_labels = {}
if nrows is not None:
sdf = sdf.limit(nrows)
index_spark_column_names: List[str]
index_names: List[Label]
if index_col is not None:
if isinstance(index_col, (str, int)):
index_col = [index_col]
for col in index_col:
if col not in column_labels:
raise KeyError(col)
index_spark_column_names = [column_labels[col] for col in index_col]
index_names = [(col,) for col in index_col]
column_labels = {
label: col for label, col in column_labels.items() if label not in index_col
}
else:
log_advice(
"If `index_col` is not specified for `read_csv`, "
"the default index is attached which can cause additional overhead."
)
index_spark_column_names = []
index_names = []
psdf: DataFrame = DataFrame(
InternalFrame(
spark_frame=sdf,
index_spark_columns=[scol_for(sdf, col) for col in index_spark_column_names],
index_names=index_names,
column_labels=[
label if is_name_like_tuple(label) else (label,) for label in column_labels
],
data_spark_columns=[scol_for(sdf, col) for col in column_labels.values()],
)
)
if dtype is not None:
if isinstance(dtype, dict):
for col, tpe in dtype.items():
psdf[col] = psdf[col].astype(tpe)
else:
for col in psdf.columns:
psdf[col] = psdf[col].astype(dtype)
return psdf
def read_json(
path: str, lines: bool = True, index_col: Optional[Union[str, List[str]]] = None, **options: Any
) -> DataFrame:
"""
Convert a JSON string to DataFrame.
Parameters
----------
path : string
File path
lines : bool, default True
Read the file as a JSON object per line. It should be always True for now.
index_col : str or list of str, optional, default: None
Index column of table in Spark.
options : dict
All other options passed directly into Spark's data source.
Examples
--------
>>> df = ps.DataFrame([['a', 'b'], ['c', 'd']],
... columns=['col 1', 'col 2'])
>>> df.to_json(path=r'%s/read_json/foo.json' % path, num_files=1)
>>> ps.read_json(
... path=r'%s/read_json/foo.json' % path
... ).sort_values(by="col 1")
col 1 col 2
0 a b
1 c d
>>> df.to_json(path=r'%s/read_json/foo.json' % path, num_files=1, lineSep='___')
>>> ps.read_json(
... path=r'%s/read_json/foo.json' % path, lineSep='___'
... ).sort_values(by="col 1")
col 1 col 2
0 a b
1 c d
You can preserve the index in the roundtrip as below.
>>> df.to_json(path=r'%s/read_json/bar.json' % path, num_files=1, index_col="index")
>>> ps.read_json(
... path=r'%s/read_json/bar.json' % path, index_col="index"
... ).sort_values(by="col 1") # doctest: +NORMALIZE_WHITESPACE
col 1 col 2
index
0 a b
1 c d
"""
if index_col is None:
log_advice(
"If `index_col` is not specified for `read_json`, "
"the default index is attached which can cause additional overhead."
)
if "options" in options and isinstance(options.get("options"), dict) and len(options) == 1:
options = options.get("options")
if not lines:
raise NotImplementedError("lines=False is not implemented yet.")
return read_spark_io(path, format="json", index_col=index_col, **options)
def read_delta(
path: str,
version: Optional[str] = None,
timestamp: Optional[str] = None,
index_col: Optional[Union[str, List[str]]] = None,
**options: Any,
) -> DataFrame:
"""
Read a Delta Lake table on some file system and return a DataFrame.
If the Delta Lake table is already stored in the catalog (aka the metastore), use 'read_table'.
Parameters
----------
path : string
Path to the Delta Lake table.
version : string, optional
Specifies the table version (based on Delta's internal transaction version) to read from,
using Delta's time travel feature. This sets Delta's 'versionAsOf' option. Note that
this parameter and `timestamp` parameter cannot be used together, otherwise it will raise a
`ValueError`.
timestamp : string, optional
Specifies the table version (based on timestamp) to read from,
using Delta's time travel feature. This must be a valid date or timestamp string in Spark,
and sets Delta's 'timestampAsOf' option. Note that this parameter and `version` parameter
cannot be used together, otherwise it will raise a `ValueError`.
index_col : str or list of str, optional, default: None
Index column of table in Spark.
options
Additional options that can be passed onto Delta.
Returns
-------
DataFrame
See Also
--------
DataFrame.to_delta
read_table
read_spark_io
read_parquet
Examples
--------
>>> ps.range(1).to_delta('%s/read_delta/foo' % path) # doctest: +SKIP
>>> ps.read_delta('%s/read_delta/foo' % path) # doctest: +SKIP
id
0 0
>>> ps.range(10, 15, num_partitions=1).to_delta('%s/read_delta/foo' % path,
... mode='overwrite') # doctest: +SKIP
>>> ps.read_delta('%s/read_delta/foo' % path) # doctest: +SKIP
id
0 10
1 11
2 12
3 13
4 14
>>> ps.read_delta('%s/read_delta/foo' % path, version=0) # doctest: +SKIP
id
0 0
You can preserve the index in the roundtrip as below.
>>> ps.range(10, 15, num_partitions=1).to_delta(
... '%s/read_delta/bar' % path, index_col="index") # doctest: +SKIP
>>> ps.read_delta('%s/read_delta/bar' % path, index_col="index") # doctest: +SKIP
id
index
0 10
1 11
2 12
3 13
4 14
"""
if index_col is None:
log_advice(
"If `index_col` is not specified for `read_delta`, "
"the default index is attached which can cause additional overhead."
)
if version is not None and timestamp is not None:
raise ValueError("version and timestamp cannot be used together.")
if "options" in options and isinstance(options.get("options"), dict) and len(options) == 1:
options = options.get("options")
if version is not None:
options["versionAsOf"] = version
if timestamp is not None:
options["timestampAsOf"] = timestamp
return read_spark_io(path, format="delta", index_col=index_col, **options)
def read_table(name: str, index_col: Optional[Union[str, List[str]]] = None) -> DataFrame:
"""
Read a Spark table and return a DataFrame.
Parameters
----------
name : string
Table name in Spark.
index_col : str or list of str, optional, default: None
Index column of table in Spark.
Returns
-------
DataFrame
See Also
--------
DataFrame.to_table
read_delta
read_parquet
read_spark_io
Examples
--------
>>> ps.range(1).to_table('%s.my_table' % db)
>>> ps.read_table('%s.my_table' % db)
id
0 0
>>> ps.range(1).to_table('%s.my_table' % db, index_col="index")
>>> ps.read_table('%s.my_table' % db, index_col="index") # doctest: +NORMALIZE_WHITESPACE
id
index
0 0
"""
if index_col is None:
log_advice(
"If `index_col` is not specified for `read_table`, "
"the default index is attached which can cause additional overhead."
)
sdf = default_session().read.table(name)
index_spark_columns, index_names = _get_index_map(sdf, index_col)
return DataFrame(
InternalFrame(
spark_frame=sdf, index_spark_columns=index_spark_columns, index_names=index_names
)
)
def read_spark_io(
path: Optional[str] = None,
format: Optional[str] = None,
schema: Union[str, "StructType"] = None,
index_col: Optional[Union[str, List[str]]] = None,
**options: Any,
) -> DataFrame:
"""Load a DataFrame from a Spark data source.
Parameters
----------
path : string, optional
Path to the data source.
format : string, optional
Specifies the output data source format. Some common ones are:
- 'delta'
- 'parquet'
- 'orc'
- 'json'
- 'csv'
schema : string or StructType, optional
Input schema. If none, Spark tries to infer the schema automatically.
The schema can either be a Spark StructType, or a DDL-formatted string like
`col0 INT, col1 DOUBLE`.
index_col : str or list of str, optional, default: None
Index column of table in Spark.
options : dict
All other options passed directly into Spark's data source.
See Also
--------
DataFrame.read_table
DataFrame.read_delta
DataFrame.read_parquet
Examples
--------
>>> ps.range(1).spark.to_spark_io('%s/read_spark_io/data.parquet' % path)
>>> ps.read_spark_io(
... '%s/read_spark_io/data.parquet' % path, format='parquet', schema='id long')
id
0 0
>>> ps.range(10, 15, num_partitions=1).spark.to_spark_io('%s/read_spark_io/data.json' % path,
... format='json', lineSep='__')
>>> ps.read_spark_io(
... '%s/read_spark_io/data.json' % path, format='json', schema='id long', lineSep='__')
id
0 10
1 11
2 12
3 13
4 14
You can preserve the index in the roundtrip as below.
>>> ps.range(10, 15, num_partitions=1).spark.to_spark_io('%s/read_spark_io/data.orc' % path,
... format='orc', index_col="index")
>>> ps.read_spark_io(
... path=r'%s/read_spark_io/data.orc' % path, format="orc", index_col="index")
... # doctest: +NORMALIZE_WHITESPACE
id
index
0 10
1 11
2 12
3 13
4 14
"""
if "options" in options and isinstance(options.get("options"), dict) and len(options) == 1:
options = options.get("options")
sdf = default_session().read.load(path=path, format=format, schema=schema, **options)
index_spark_columns, index_names = _get_index_map(sdf, index_col)
return DataFrame(
InternalFrame(
spark_frame=sdf, index_spark_columns=index_spark_columns, index_names=index_names
)
)
def read_parquet(
path: str,
columns: Optional[List[str]] = None,
index_col: Optional[List[str]] = None,
pandas_metadata: bool = False,
**options: Any,
) -> DataFrame:
"""Load a parquet object from the file path, returning a DataFrame.
Parameters
----------
path : string
File path
columns : list, default=None
If not None, only these columns will be read from the file.
index_col : str or list of str, optional, default: None
Index column of table in Spark.
pandas_metadata : bool, default: False
If True, try to respect the metadata if the Parquet file is written from pandas.
options : dict
All other options passed directly into Spark's data source.
Returns
-------
DataFrame
See Also
--------
DataFrame.to_parquet
DataFrame.read_table
DataFrame.read_delta
DataFrame.read_spark_io
Examples
--------
>>> ps.range(1).to_parquet('%s/read_spark_io/data.parquet' % path)
>>> ps.read_parquet('%s/read_spark_io/data.parquet' % path, columns=['id'])
id
0 0
You can preserve the index in the roundtrip as below.
>>> ps.range(1).to_parquet('%s/read_spark_io/data.parquet' % path, index_col="index")
>>> ps.read_parquet('%s/read_spark_io/data.parquet' % path, columns=['id'], index_col="index")
... # doctest: +NORMALIZE_WHITESPACE
id
index
0 0
"""
if index_col is None:
log_advice(
"If `index_col` is not specified for `read_parquet`, "
"the default index is attached which can cause additional overhead."
)
if "options" in options and isinstance(options.get("options"), dict) and len(options) == 1:
options = options.get("options")
if columns is not None:
columns = list(columns)
index_names = None
if index_col is None and pandas_metadata:
# Try to read pandas metadata
@pandas_udf( # type: ignore[call-overload]
"index_col array<string>, index_names array<string>"
)
def read_index_metadata(pser: pd.Series) -> pd.DataFrame:
binary = pser.iloc[0]
metadata = pq.ParquetFile(pa.BufferReader(binary)).metadata.metadata
if b"pandas" in metadata:
pandas_metadata = json.loads(metadata[b"pandas"].decode("utf8"))
if all(isinstance(col, str) for col in pandas_metadata["index_columns"]):
index_col = []
index_names = []
for col in pandas_metadata["index_columns"]:
index_col.append(col)
for column in pandas_metadata["columns"]:
if column["field_name"] == col:
index_names.append(column["name"])
break
else:
index_names.append(None)
return pd.DataFrame({"index_col": [index_col], "index_names": [index_names]})
return pd.DataFrame({"index_col": [None], "index_names": [None]})
index_col, index_names = (
default_session()
.read.format("binaryFile")
.load(path)
.limit(1)
.select(read_index_metadata("content").alias("index_metadata"))
.select("index_metadata.*")
.head()
)
psdf = read_spark_io(path=path, format="parquet", options=options, index_col=index_col)
if columns is not None:
new_columns = [c for c in columns if c in psdf.columns]
if len(new_columns) > 0:
psdf = psdf[new_columns]
else:
sdf = default_session().createDataFrame([], schema=StructType())
index_spark_columns, index_names = _get_index_map(sdf, index_col)
psdf = DataFrame(
InternalFrame(
spark_frame=sdf,
index_spark_columns=index_spark_columns,
index_names=index_names,
)
)
if index_names is not None:
psdf.index.names = index_names
return psdf
def read_clipboard(sep: str = r"\s+", **kwargs: Any) -> DataFrame:
r"""
Read text from clipboard and pass to read_csv. See read_csv for the
full argument list
Parameters
----------
sep : str, default '\s+'
A string or regex delimiter. The default of '\s+' denotes
one or more whitespace characters.
See Also
--------
DataFrame.to_clipboard : Write text out to clipboard.
Returns
-------
parsed : DataFrame
"""
return cast(DataFrame, from_pandas(pd.read_clipboard(sep, **kwargs)))
def read_excel(
io: Union[str, Any],
sheet_name: Union[str, int, List[Union[str, int]], None] = 0,
header: Union[int, List[int]] = 0,
names: Optional[List] = None,
index_col: Optional[List[int]] = None,
usecols: Optional[Union[int, str, List[Union[int, str]], Callable[[str], bool]]] = None,
dtype: Optional[Dict[str, Union[str, Dtype]]] = None,
engine: Optional[str] = None,
converters: Optional[Dict] = None,
true_values: Optional[Any] = None,
false_values: Optional[Any] = None,
skiprows: Optional[Union[int, List[int]]] = None,
nrows: Optional[int] = None,
na_values: Optional[Any] = None,
keep_default_na: bool = True,
verbose: bool = False,
parse_dates: Union[bool, List, Dict] = False,
date_parser: Optional[Callable] = None,
thousands: Optional[str] = None,
comment: Optional[str] = None,
skipfooter: int = 0,
**kwds: Any,
) -> Union[DataFrame, Series, Dict[str, Union[DataFrame, Series]]]:
"""
Read an Excel file into a pandas-on-Spark DataFrame or Series.
Support both `xls` and `xlsx` file extensions from a local filesystem or URL.
Support an option to read a single sheet or a list of sheets.
Parameters
----------
io : str, file descriptor, pathlib.Path, ExcelFile or xlrd.Book
The string could be a URL. The value URL must be available in Spark's DataFrameReader.
.. note::
If the underlying Spark is below 3.0, the parameter as a string is not supported.
You can use `ps.from_pandas(pd.read_excel(...))` as a workaround.
sheet_name : str, int, list, or None, default 0
Strings are used for sheet names. Integers are used in zero-indexed
sheet positions. Lists of strings/integers are used to request
multiple sheets. Specify None to get all sheets.
Available cases:
* Defaults to ``0``: 1st sheet as a `DataFrame`
* ``1``: 2nd sheet as a `DataFrame`
* ``"Sheet1"``: Load sheet with name "Sheet1"
* ``[0, 1, "Sheet5"]``: Load first, second and sheet named "Sheet5"
as a dict of `DataFrame`
* None: All sheets.
header : int, list of int, default 0
Row (0-indexed) to use for the column labels of the parsed
DataFrame. If a list of integers is passed those row positions will
be combined into a ``MultiIndex``. Use None if there is no header.
names : array-like, default None
List of column names to use. If file contains no header row,
then you should explicitly pass header=None.
index_col : int, list of int, default None
Column (0-indexed) to use as the row labels of the DataFrame.
Pass None if there is no such column. If a list is passed,
those columns will be combined into a ``MultiIndex``. If a
subset of data is selected with ``usecols``, index_col
is based on the subset.
usecols : int, str, list-like, or callable default None
Return a subset of the columns.
* If None, then parse all columns.
* If str, then indicates comma separated list of Excel column letters
and column ranges (e.g. "A:E" or "A,C,E:F"). Ranges are inclusive of
both sides.
* If list of int, then indicates list of column numbers to be parsed.
* If list of string, then indicates list of column names to be parsed.
* If callable, then evaluate each column name against it and parse the
column if the callable returns ``True``.
dtype : Type name or dict of column -> type, default None
Data type for data or columns. E.g. {'a': np.float64, 'b': np.int32}
Use `object` to preserve data as stored in Excel and not interpret dtype.
If converters are specified, they will be applied INSTEAD
of dtype conversion.
engine : str, default None