-
Notifications
You must be signed in to change notification settings - Fork 28.5k
/
Copy pathtypes.py
3337 lines (2789 loc) · 110 KB
/
types.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 os
import sys
import decimal
import time
import math
import datetime
import calendar
import json
import re
import base64
from array import array
import ctypes
from collections.abc import Iterable
from functools import reduce
from typing import (
cast,
overload,
Any,
Callable,
ClassVar,
Dict,
Iterator,
List,
Optional,
Union,
Tuple,
Type,
TypeVar,
TYPE_CHECKING,
)
from pyspark.util import is_remote_only, JVM_INT_MAX
from pyspark.serializers import CloudPickleSerializer
from pyspark.sql.utils import (
get_active_spark_context,
escape_meta_characters,
StringConcat,
)
from pyspark.sql.variant_utils import VariantUtils
from pyspark.errors import (
PySparkNotImplementedError,
PySparkTypeError,
PySparkValueError,
PySparkIndexError,
PySparkRuntimeError,
PySparkAttributeError,
PySparkKeyError,
)
if TYPE_CHECKING:
import numpy as np
from py4j.java_gateway import GatewayClient, JavaGateway, JavaClass
T = TypeVar("T")
U = TypeVar("U")
__all__ = [
"DataType",
"NullType",
"CharType",
"StringType",
"VarcharType",
"BinaryType",
"BooleanType",
"DateType",
"TimestampType",
"TimestampNTZType",
"DecimalType",
"DoubleType",
"FloatType",
"ByteType",
"IntegerType",
"LongType",
"DayTimeIntervalType",
"YearMonthIntervalType",
"CalendarIntervalType",
"Row",
"ShortType",
"ArrayType",
"MapType",
"StructField",
"StructType",
"VariantType",
"VariantVal",
]
class DataType:
"""Base class for data types."""
def __repr__(self) -> str:
return self.__class__.__name__ + "()"
def __hash__(self) -> int:
return hash(str(self))
def __eq__(self, other: Any) -> bool:
return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
def __ne__(self, other: Any) -> bool:
return not self.__eq__(other)
@classmethod
def typeName(cls) -> str:
return cls.__name__[:-4].lower()
def simpleString(self) -> str:
return self.typeName()
def jsonValue(self) -> Union[str, Dict[str, Any]]:
return self.typeName()
def json(self) -> str:
return json.dumps(self.jsonValue(), separators=(",", ":"), sort_keys=True)
def needConversion(self) -> bool:
"""
Does this type needs conversion between Python object and internal SQL object.
This is used to avoid the unnecessary conversion for ArrayType/MapType/StructType.
"""
return False
def toInternal(self, obj: Any) -> Any:
"""
Converts a Python object into an internal SQL object.
"""
return obj
def fromInternal(self, obj: Any) -> Any:
"""
Converts an internal SQL object into a native Python object.
"""
return obj
def _as_nullable(self) -> "DataType":
return self
@classmethod
def fromDDL(cls, ddl: str) -> "DataType":
"""
Creates :class:`DataType` for a given DDL-formatted string.
.. versionadded:: 4.0.0
Parameters
----------
ddl : str
DDL-formatted string representation of types, e.g.
:class:`pyspark.sql.types.DataType.simpleString`, except that top level struct
type can omit the ``struct<>`` for the compatibility reason with
``spark.createDataFrame`` and Python UDFs.
Returns
-------
:class:`DataType`
Examples
--------
Create a StructType by the corresponding DDL formatted string.
>>> from pyspark.sql.types import DataType
>>> DataType.fromDDL("b string, a int")
StructType([StructField('b', StringType(), True), StructField('a', IntegerType(), True)])
Create a single DataType by the corresponding DDL formatted string.
>>> DataType.fromDDL("decimal(10,10)")
DecimalType(10,10)
Create a StructType by the legacy string format.
>>> DataType.fromDDL("b: string, a: int")
StructType([StructField('b', StringType(), True), StructField('a', IntegerType(), True)])
"""
return _parse_datatype_string(ddl)
@classmethod
def _data_type_build_formatted_string(
cls,
dataType: "DataType",
prefix: str,
stringConcat: StringConcat,
maxDepth: int,
) -> None:
if isinstance(dataType, (ArrayType, StructType, MapType)):
dataType._build_formatted_string(prefix, stringConcat, maxDepth - 1)
# The method typeName() is not always the same as the Scala side.
# Add this helper method to make TreeString() compatible with Scala side.
@classmethod
def _get_jvm_type_name(cls, dataType: "DataType") -> str:
if isinstance(
dataType,
(
DecimalType,
CharType,
VarcharType,
DayTimeIntervalType,
YearMonthIntervalType,
),
):
return dataType.simpleString()
else:
return dataType.typeName()
# This singleton pattern does not work with pickle, you will get
# another object after pickle and unpickle
class DataTypeSingleton(type):
"""Metaclass for DataType"""
_instances: ClassVar[Dict[Type["DataTypeSingleton"], "DataTypeSingleton"]] = {}
def __call__(cls: Type[T]) -> T:
if cls not in cls._instances: # type: ignore[attr-defined]
cls._instances[cls] = super( # type: ignore[misc, attr-defined]
DataTypeSingleton, cls
).__call__()
return cls._instances[cls] # type: ignore[attr-defined]
class NullType(DataType, metaclass=DataTypeSingleton):
"""Null type.
The data type representing None, used for the types that cannot be inferred.
"""
@classmethod
def typeName(cls) -> str:
return "void"
class AtomicType(DataType):
"""An internal type used to represent everything that is not
null, UDTs, arrays, structs, and maps."""
class NumericType(AtomicType):
"""Numeric data types."""
class IntegralType(NumericType, metaclass=DataTypeSingleton):
"""Integral data types."""
pass
class FractionalType(NumericType):
"""Fractional data types."""
class StringType(AtomicType):
"""String data type.
Parameters
----------
collation : str
name of the collation, default is UTF8_BINARY.
"""
providerSpark = "spark"
providerICU = "icu"
providers = [providerSpark, providerICU]
def __init__(self, collation: str = "UTF8_BINARY"):
self.collation = collation
@classmethod
def collationProvider(cls, collationName: str) -> str:
# TODO: do this properly like on the scala side
if collationName.startswith("UTF8"):
return StringType.providerSpark
return StringType.providerICU
def simpleString(self) -> str:
if self.isUTF8BinaryCollation():
return "string"
return f"string collate {self.collation}"
# For backwards compatibility and compatibility with other readers all string types
# are serialized in json as regular strings and the collation info is written to
# struct field metadata
def jsonValue(self) -> str:
return "string"
def __repr__(self) -> str:
return (
"StringType()" if self.isUTF8BinaryCollation() else "StringType('%s')" % self.collation
)
def isUTF8BinaryCollation(self) -> bool:
return self.collation == "UTF8_BINARY"
class CharType(AtomicType):
"""Char data type
Parameters
----------
length : int
the length limitation.
"""
def __init__(self, length: int):
self.length = length
def simpleString(self) -> str:
return "char(%d)" % (self.length)
def jsonValue(self) -> str:
return "char(%d)" % (self.length)
def __repr__(self) -> str:
return "CharType(%d)" % (self.length)
class VarcharType(AtomicType):
"""Varchar data type
Parameters
----------
length : int
the length limitation.
"""
def __init__(self, length: int):
self.length = length
def simpleString(self) -> str:
return "varchar(%d)" % (self.length)
def jsonValue(self) -> str:
return "varchar(%d)" % (self.length)
def __repr__(self) -> str:
return "VarcharType(%d)" % (self.length)
class BinaryType(AtomicType, metaclass=DataTypeSingleton):
"""Binary (byte array) data type."""
pass
class BooleanType(AtomicType, metaclass=DataTypeSingleton):
"""Boolean data type."""
pass
class DateType(AtomicType, metaclass=DataTypeSingleton):
"""Date (datetime.date) data type."""
EPOCH_ORDINAL = datetime.datetime(1970, 1, 1).toordinal()
def needConversion(self) -> bool:
return True
def toInternal(self, d: datetime.date) -> int:
if d is not None:
return d.toordinal() - self.EPOCH_ORDINAL
def fromInternal(self, v: int) -> datetime.date:
if v is not None:
return datetime.date.fromordinal(v + self.EPOCH_ORDINAL)
class TimestampType(AtomicType, metaclass=DataTypeSingleton):
"""Timestamp (datetime.datetime) data type."""
def needConversion(self) -> bool:
return True
def toInternal(self, dt: datetime.datetime) -> int:
if dt is not None:
seconds = (
calendar.timegm(dt.utctimetuple()) if dt.tzinfo else time.mktime(dt.timetuple())
)
return int(seconds) * 1000000 + dt.microsecond
def fromInternal(self, ts: int) -> datetime.datetime:
if ts is not None:
# using int to avoid precision loss in float
return datetime.datetime.fromtimestamp(ts // 1000000).replace(microsecond=ts % 1000000)
class TimestampNTZType(AtomicType, metaclass=DataTypeSingleton):
"""Timestamp (datetime.datetime) data type without timezone information."""
def needConversion(self) -> bool:
return True
@classmethod
def typeName(cls) -> str:
return "timestamp_ntz"
def toInternal(self, dt: datetime.datetime) -> int:
if dt is not None:
seconds = calendar.timegm(dt.timetuple())
return int(seconds) * 1000000 + dt.microsecond
def fromInternal(self, ts: int) -> datetime.datetime:
if ts is not None:
# using int to avoid precision loss in float
return datetime.datetime.fromtimestamp(ts // 1000000, datetime.timezone.utc).replace(
microsecond=ts % 1000000, tzinfo=None
)
class DecimalType(FractionalType):
"""Decimal (decimal.Decimal) data type.
The DecimalType must have fixed precision (the maximum total number of digits)
and scale (the number of digits on the right of dot). For example, (5, 2) can
support the value from [-999.99 to 999.99].
The precision can be up to 38, the scale must be less or equal to precision.
When creating a DecimalType, the default precision and scale is (10, 0). When inferring
schema from decimal.Decimal objects, it will be DecimalType(38, 18).
Parameters
----------
precision : int, optional
the maximum (i.e. total) number of digits (default: 10)
scale : int, optional
the number of digits on right side of dot. (default: 0)
"""
def __init__(self, precision: int = 10, scale: int = 0):
self.precision = precision
self.scale = scale
self.hasPrecisionInfo = True # this is a public API
def simpleString(self) -> str:
return "decimal(%d,%d)" % (self.precision, self.scale)
def jsonValue(self) -> str:
return "decimal(%d,%d)" % (self.precision, self.scale)
def __repr__(self) -> str:
return "DecimalType(%d,%d)" % (self.precision, self.scale)
class DoubleType(FractionalType, metaclass=DataTypeSingleton):
"""Double data type, representing double precision floats."""
pass
class FloatType(FractionalType, metaclass=DataTypeSingleton):
"""Float data type, representing single precision floats."""
pass
class ByteType(IntegralType):
"""Byte data type, representing signed 8-bit integers."""
def simpleString(self) -> str:
return "tinyint"
class IntegerType(IntegralType):
"""Int data type, representing signed 32-bit integers."""
def simpleString(self) -> str:
return "int"
class LongType(IntegralType):
"""Long data type, representing signed 64-bit integers.
If the values are beyond the range of [-9223372036854775808, 9223372036854775807],
please use :class:`DecimalType`.
"""
def simpleString(self) -> str:
return "bigint"
class ShortType(IntegralType):
"""Short data type, representing signed 16-bit integers."""
def simpleString(self) -> str:
return "smallint"
class AnsiIntervalType(AtomicType):
"""The interval type which conforms to the ANSI SQL standard."""
pass
class DayTimeIntervalType(AnsiIntervalType):
"""DayTimeIntervalType (datetime.timedelta)."""
DAY = 0
HOUR = 1
MINUTE = 2
SECOND = 3
_fields = {
DAY: "day",
HOUR: "hour",
MINUTE: "minute",
SECOND: "second",
}
_inverted_fields = dict(zip(_fields.values(), _fields.keys()))
def __init__(self, startField: Optional[int] = None, endField: Optional[int] = None):
if startField is None and endField is None:
# Default matched to scala side.
startField = DayTimeIntervalType.DAY
endField = DayTimeIntervalType.SECOND
elif startField is not None and endField is None:
endField = startField
fields = DayTimeIntervalType._fields
if startField not in fields.keys() or endField not in fields.keys():
raise PySparkRuntimeError(
errorClass="INVALID_INTERVAL_CASTING",
messageParameters={"start_field": str(startField), "end_field": str(endField)},
)
self.startField = startField
self.endField = endField
def _str_repr(self) -> str:
fields = DayTimeIntervalType._fields
start_field_name = fields[self.startField]
end_field_name = fields[self.endField]
if start_field_name == end_field_name:
return "interval %s" % start_field_name
else:
return "interval %s to %s" % (start_field_name, end_field_name)
simpleString = _str_repr
jsonValue = _str_repr
def __repr__(self) -> str:
return "%s(%d, %d)" % (type(self).__name__, self.startField, self.endField)
def needConversion(self) -> bool:
return True
def toInternal(self, dt: datetime.timedelta) -> Optional[int]:
if dt is not None:
return (((dt.days * 86400) + dt.seconds) * 1_000_000) + dt.microseconds
def fromInternal(self, micros: int) -> Optional[datetime.timedelta]:
if micros is not None:
return datetime.timedelta(microseconds=micros)
class YearMonthIntervalType(AnsiIntervalType):
"""YearMonthIntervalType, represents year-month intervals of the SQL standard
Notes
-----
This data type doesn't support collection: df.collect/take/head.
"""
YEAR = 0
MONTH = 1
_fields = {
YEAR: "year",
MONTH: "month",
}
_inverted_fields = dict(zip(_fields.values(), _fields.keys()))
def __init__(self, startField: Optional[int] = None, endField: Optional[int] = None):
if startField is None and endField is None:
# Default matched to scala side.
startField = YearMonthIntervalType.YEAR
endField = YearMonthIntervalType.MONTH
elif startField is not None and endField is None:
endField = startField
fields = YearMonthIntervalType._fields
if startField not in fields.keys() or endField not in fields.keys():
raise PySparkRuntimeError(
errorClass="INVALID_INTERVAL_CASTING",
messageParameters={"start_field": str(startField), "end_field": str(endField)},
)
self.startField = startField
self.endField = endField
def _str_repr(self) -> str:
fields = YearMonthIntervalType._fields
start_field_name = fields[self.startField]
end_field_name = fields[self.endField]
if start_field_name == end_field_name:
return "interval %s" % start_field_name
else:
return "interval %s to %s" % (start_field_name, end_field_name)
simpleString = _str_repr
jsonValue = _str_repr
def needConversion(self) -> bool:
# If PYSPARK_YM_INTERVAL_LEGACY is not set, needConversion is true,
# 'df.collect' fails with PySparkNotImplementedError;
# otherwise, no conversion is needed, and 'df.collect' returns the internal integers.
return not os.environ.get("PYSPARK_YM_INTERVAL_LEGACY") == "1"
def toInternal(self, obj: Any) -> Any:
raise PySparkNotImplementedError(
errorClass="NOT_IMPLEMENTED",
messageParameters={"feature": "YearMonthIntervalType.toInternal"},
)
def fromInternal(self, obj: Any) -> Any:
raise PySparkNotImplementedError(
errorClass="NOT_IMPLEMENTED",
messageParameters={"feature": "YearMonthIntervalType.fromInternal"},
)
def __repr__(self) -> str:
return "%s(%d, %d)" % (type(self).__name__, self.startField, self.endField)
class CalendarIntervalType(DataType, metaclass=DataTypeSingleton):
"""The data type representing calendar intervals.
The calendar interval is stored internally in three components:
- an integer value representing the number of `months` in this interval.
- an integer value representing the number of `days` in this interval.
- a long value representing the number of `microseconds` in this interval.
"""
@classmethod
def typeName(cls) -> str:
return "interval"
def needConversion(self) -> bool:
return True
def toInternal(self, obj: Any) -> Any:
raise PySparkNotImplementedError(
errorClass="NOT_IMPLEMENTED",
messageParameters={"feature": "CalendarIntervalType.toInternal"},
)
def fromInternal(self, obj: Any) -> Any:
raise PySparkNotImplementedError(
errorClass="NOT_IMPLEMENTED",
messageParameters={"feature": "CalendarIntervalType.fromInternal"},
)
class ArrayType(DataType):
"""Array data type.
Parameters
----------
elementType : :class:`DataType`
:class:`DataType` of each element in the array.
containsNull : bool, optional
whether the array can contain null (None) values.
Examples
--------
>>> from pyspark.sql.types import ArrayType, StringType, StructField, StructType
The below example demonstrates how to create class:`ArrayType`:
>>> arr = ArrayType(StringType())
The array can contain null (None) values by default:
>>> ArrayType(StringType()) == ArrayType(StringType(), True)
True
>>> ArrayType(StringType(), False) == ArrayType(StringType())
False
"""
def __init__(self, elementType: DataType, containsNull: bool = True):
assert isinstance(elementType, DataType), "elementType %s should be an instance of %s" % (
elementType,
DataType,
)
self.elementType = elementType
self.containsNull = containsNull
def simpleString(self) -> str:
return "array<%s>" % self.elementType.simpleString()
def _as_nullable(self) -> "ArrayType":
return ArrayType(self.elementType._as_nullable(), containsNull=True)
def toNullable(self) -> "ArrayType":
"""
Returns the same data type but set all nullability fields are true
(`StructField.nullable`, `ArrayType.containsNull`, and `MapType.valueContainsNull`).
.. versionadded:: 4.0.0
Returns
-------
:class:`ArrayType`
Examples
--------
Example 1: Simple nullability conversion
>>> ArrayType(IntegerType(), containsNull=False).toNullable()
ArrayType(IntegerType(), True)
Example 2: Nested nullability conversion
>>> ArrayType(
... StructType([
... StructField("b", IntegerType(), nullable=False),
... StructField("c", ArrayType(IntegerType(), containsNull=False))
... ]),
... containsNull=False
... ).toNullable()
ArrayType(StructType([StructField('b', IntegerType(), True),
StructField('c', ArrayType(IntegerType(), True), True)]), True)
"""
return self._as_nullable()
def __repr__(self) -> str:
return "ArrayType(%s, %s)" % (self.elementType, str(self.containsNull))
def jsonValue(self) -> Dict[str, Any]:
return {
"type": self.typeName(),
"elementType": self.elementType.jsonValue(),
"containsNull": self.containsNull,
}
@classmethod
def fromJson(
cls,
json: Dict[str, Any],
fieldPath: str = "",
collationsMap: Optional[Dict[str, str]] = None,
) -> "ArrayType":
elementType = _parse_datatype_json_value(
json["elementType"],
"element" if fieldPath == "" else fieldPath + ".element",
collationsMap,
)
return ArrayType(elementType, json["containsNull"])
def needConversion(self) -> bool:
return self.elementType.needConversion()
def toInternal(self, obj: List[Optional[T]]) -> List[Optional[T]]:
if not self.needConversion():
return obj
return obj and [self.elementType.toInternal(v) for v in obj]
def fromInternal(self, obj: List[Optional[T]]) -> List[Optional[T]]:
if not self.needConversion():
return obj
return obj and [self.elementType.fromInternal(v) for v in obj]
def _build_formatted_string(
self,
prefix: str,
stringConcat: StringConcat,
maxDepth: int = JVM_INT_MAX,
) -> None:
if maxDepth > 0:
stringConcat.append(
f"{prefix}-- element: {DataType._get_jvm_type_name(self.elementType)} "
+ f"(containsNull = {str(self.containsNull).lower()})\n"
)
DataType._data_type_build_formatted_string(
self.elementType, f"{prefix} |", stringConcat, maxDepth
)
class MapType(DataType):
"""Map data type.
Parameters
----------
keyType : :class:`DataType`
:class:`DataType` of the keys in the map.
valueType : :class:`DataType`
:class:`DataType` of the values in the map.
valueContainsNull : bool, optional
indicates whether values can contain null (None) values.
Notes
-----
Keys in a map data type are not allowed to be null (None).
Examples
--------
>>> from pyspark.sql.types import IntegerType, FloatType, MapType, StringType
The below example demonstrates how to create class:`MapType`:
>>> map_type = MapType(StringType(), IntegerType())
The values of the map can contain null (``None``) values by default:
>>> (MapType(StringType(), IntegerType())
... == MapType(StringType(), IntegerType(), True))
True
>>> (MapType(StringType(), IntegerType(), False)
... == MapType(StringType(), FloatType()))
False
"""
def __init__(self, keyType: DataType, valueType: DataType, valueContainsNull: bool = True):
assert isinstance(keyType, DataType), "keyType %s should be an instance of %s" % (
keyType,
DataType,
)
assert isinstance(valueType, DataType), "valueType %s should be an instance of %s" % (
valueType,
DataType,
)
self.keyType = keyType
self.valueType = valueType
self.valueContainsNull = valueContainsNull
def simpleString(self) -> str:
return "map<%s,%s>" % (self.keyType.simpleString(), self.valueType.simpleString())
def _as_nullable(self) -> "MapType":
return MapType(
self.keyType._as_nullable(), self.valueType._as_nullable(), valueContainsNull=True
)
def toNullable(self) -> "MapType":
"""
Returns the same data type but set all nullability fields are true
(`StructField.nullable`, `ArrayType.containsNull`, and `MapType.valueContainsNull`).
.. versionadded:: 4.0.0
Returns
-------
:class:`MapType`
Examples
--------
Example 1: Simple nullability conversion
>>> MapType(IntegerType(), StringType(), valueContainsNull=False).toNullable()
MapType(IntegerType(), StringType(), True)
Example 2: Nested nullability conversion
>>> MapType(
... StringType(),
... MapType(
... IntegerType(),
... ArrayType(IntegerType(), containsNull=False),
... valueContainsNull=False
... ),
... valueContainsNull=False
... ).toNullable()
MapType(StringType(), MapType(IntegerType(), ArrayType(IntegerType(), True), True), True)
"""
return self._as_nullable()
def __repr__(self) -> str:
return "MapType(%s, %s, %s)" % (self.keyType, self.valueType, str(self.valueContainsNull))
def jsonValue(self) -> Dict[str, Any]:
return {
"type": self.typeName(),
"keyType": self.keyType.jsonValue(),
"valueType": self.valueType.jsonValue(),
"valueContainsNull": self.valueContainsNull,
}
@classmethod
def fromJson(
cls,
json: Dict[str, Any],
fieldPath: str = "",
collationsMap: Optional[Dict[str, str]] = None,
) -> "MapType":
keyType = _parse_datatype_json_value(
json["keyType"], "key" if fieldPath == "" else fieldPath + ".key", collationsMap
)
valueType = _parse_datatype_json_value(
json["valueType"], "value" if fieldPath == "" else fieldPath + ".value", collationsMap
)
return MapType(
keyType,
valueType,
json["valueContainsNull"],
)
def needConversion(self) -> bool:
return self.keyType.needConversion() or self.valueType.needConversion()
def toInternal(self, obj: Dict[T, Optional[U]]) -> Dict[T, Optional[U]]:
if not self.needConversion():
return obj
return obj and dict(
(self.keyType.toInternal(k), self.valueType.toInternal(v)) for k, v in obj.items()
)
def fromInternal(self, obj: Dict[T, Optional[U]]) -> Dict[T, Optional[U]]:
if not self.needConversion():
return obj
return obj and dict(
(self.keyType.fromInternal(k), self.valueType.fromInternal(v)) for k, v in obj.items()
)
def _build_formatted_string(
self,
prefix: str,
stringConcat: StringConcat,
maxDepth: int = JVM_INT_MAX,
) -> None:
if maxDepth > 0:
stringConcat.append(f"{prefix}-- key: {DataType._get_jvm_type_name(self.keyType)}\n")
DataType._data_type_build_formatted_string(
self.keyType, f"{prefix} |", stringConcat, maxDepth
)
stringConcat.append(
f"{prefix}-- value: {DataType._get_jvm_type_name(self.valueType)} "
+ f"(valueContainsNull = {str(self.valueContainsNull).lower()})\n"
)
DataType._data_type_build_formatted_string(
self.valueType, f"{prefix} |", stringConcat, maxDepth
)
class StructField(DataType):
"""A field in :class:`StructType`.
Parameters
----------
name : str
name of the field.
dataType : :class:`DataType`
:class:`DataType` of the field.
nullable : bool, optional
whether the field can be null (None) or not.
metadata : dict, optional
a dict from string to simple type that can be toInternald to JSON automatically
Examples
--------
>>> from pyspark.sql.types import StringType, StructField
>>> (StructField("f1", StringType(), True)
... == StructField("f1", StringType(), True))
True
>>> (StructField("f1", StringType(), True)
... == StructField("f2", StringType(), True))
False
"""
def __init__(
self,
name: str,
dataType: DataType,
nullable: bool = True,
metadata: Optional[Dict[str, Any]] = None,
):
assert isinstance(dataType, DataType), "dataType %s should be an instance of %s" % (
dataType,
DataType,
)
assert isinstance(name, str), "field name %s should be a string" % (name)
self.name = name
self.dataType = dataType
self.nullable = nullable
self.metadata = metadata or {}
def simpleString(self) -> str:
return "%s:%s" % (self.name, self.dataType.simpleString())
def __repr__(self) -> str: