-
Notifications
You must be signed in to change notification settings - Fork 13.5k
/
Copy pathtypes.py
2814 lines (2303 loc) · 108 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 calendar
import ctypes
import datetime
import decimal
import sys
import time
from array import array
from copy import copy
from enum import Enum
from functools import reduce
from threading import RLock
from py4j.java_gateway import get_java_class
from typing import List, Union
from pyflink.common.types import _create_row
from pyflink.util.java_utils import to_jarray, is_instance_of
from pyflink.java_gateway import get_gateway
from pyflink.common import Row, RowKind
__all__ = ['DataTypes', 'UserDefinedType', 'Row', 'RowKind']
class DataType(object):
"""
Describes the data type of a value in the table ecosystem. Instances of this class can be used
to declare input and/or output types of operations.
:class:`DataType` has two responsibilities: declaring a logical type and giving hints
about the physical representation of data to the optimizer. While the logical type is mandatory,
hints are optional but useful at the edges to other APIs.
The logical type is independent of any physical representation and is close to the "data type"
terminology of the SQL standard.
Physical hints are required at the edges of the table ecosystem. Hints indicate the data format
that an implementation expects.
:param nullable: boolean, whether the type can be null (None) or not.
"""
def __init__(self, nullable=True):
self._nullable = nullable
self._conversion_cls = ''
def __repr__(self):
return '%s(%s)' % (self.__class__.__name__, str(self._nullable).lower())
def __str__(self, *args, **kwargs):
return self.__class__.type_name()
def __hash__(self):
return hash(str(self))
def __eq__(self, other):
return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
def __ne__(self, other):
return not self.__eq__(other)
def not_null(self):
cp = copy(self)
cp._nullable = False
return cp
def nullable(self):
cp = copy(self)
cp._nullable = True
return cp
@classmethod
def type_name(cls) -> str:
return cls.__name__[:-4].upper()
def bridged_to(self, conversion_cls) -> 'DataType':
"""
Adds a hint that data should be represented using the given class when entering or leaving
the table ecosystem.
:param conversion_cls: the string representation of the conversion class
"""
self._conversion_cls = conversion_cls
return self
def need_conversion(self) -> bool:
"""
Does this type need to conversion between Python object and internal SQL object.
This is used to avoid the unnecessary conversion for ArrayType/MultisetType/MapType/RowType.
"""
return False
def to_sql_type(self, obj):
"""
Converts a Python object into an internal SQL object.
"""
return obj
def from_sql_type(self, obj):
"""
Converts an internal SQL object into a native Python object.
"""
return obj
class AtomicType(DataType):
"""
An internal type used to represent everything that is not
arrays, rows, and maps.
"""
def __init__(self, nullable=True):
super(AtomicType, self).__init__(nullable)
class NullType(AtomicType):
"""
Null type.
The data type representing None.
"""
def __init__(self):
super(NullType, self).__init__(True)
class NumericType(AtomicType):
"""
Numeric data types.
"""
def __init__(self, nullable=True):
super(NumericType, self).__init__(nullable)
class IntegralType(NumericType):
"""
Integral data types.
"""
def __init__(self, nullable=True):
super(IntegralType, self).__init__(nullable)
class FractionalType(NumericType):
"""
Fractional data types.
"""
def __init__(self, nullable=True):
super(FractionalType, self).__init__(nullable)
class CharType(AtomicType):
"""
Char data type. SQL CHAR(n)
The serialized string representation is ``char(n)`` where ``n`` (default: 1) is the number of
code points. ``n`` must have a value between 1 and 2147483647(0x7fffffff) (both inclusive).
:param length: int, the string representation length.
:param nullable: boolean, whether the type can be null (None) or not.
"""
def __init__(self, length=1, nullable=True):
super(CharType, self).__init__(nullable)
self.length = length
def __repr__(self):
return 'CharType(%d, %s)' % (self.length, str(self._nullable).lower())
class VarCharType(AtomicType):
"""
Varchar data type. SQL VARCHAR(n)
The serialized string representation is ``varchar(n)`` where 'n' (default: 1) is the maximum
number of code points. 'n' must have a value between 1 and 2147483647(0x7fffffff)
(both inclusive).
:param length: int, the maximum string representation length.
:param nullable: boolean, whether the type can be null (None) or not.
"""
def __init__(self, length=1, nullable=True):
super(VarCharType, self).__init__(nullable)
self.length = length
def __repr__(self):
return "VarCharType(%d, %s)" % (self.length, str(self._nullable).lower())
class BinaryType(AtomicType):
"""
Binary (byte array) data type. SQL BINARY(n)
The serialized string representation is ``binary(n)`` where ``n`` (default: 1) is the number of
bytes. ``n`` must have a value between 1 and 2147483647(0x7fffffff) (both inclusive).
:param length: int, the number of bytes.
:param nullable: boolean, whether the type can be null (None) or not.
"""
def __init__(self, length=1, nullable=True):
super(BinaryType, self).__init__(nullable)
self.length = length
def __repr__(self):
return "BinaryType(%d, %s)" % (self.length, str(self._nullable).lower())
class VarBinaryType(AtomicType):
"""
Binary (byte array) data type. SQL VARBINARY(n)
The serialized string representation is ``varbinary(n)`` where ``n`` (default: 1) is the
maximum number of bytes. ``n`` must have a value between 1 and 0x7fffffff (both inclusive).
:param length: int, the maximum number of bytes.
:param nullable: boolean, whether the type can be null (None) or not.
"""
def __init__(self, length=1, nullable=True):
super(VarBinaryType, self).__init__(nullable)
self.length = length
def __repr__(self):
return "VarBinaryType(%d, %s)" % (self.length, str(self._nullable).lower())
class BooleanType(AtomicType):
"""
Boolean data types. SQL BOOLEAN
:param nullable: boolean, whether the field can be null (None) or not.
"""
def __init__(self, nullable=True):
super(BooleanType, self).__init__(nullable)
class TinyIntType(IntegralType):
"""
Byte data type. SQL TINYINT (8bits)
:param nullable: boolean, whether the field can be null (None) or not.
"""
def __init__(self, nullable=True):
super(TinyIntType, self).__init__(nullable)
class SmallIntType(IntegralType):
"""
Short data type. SQL SMALLINT (16bits)
:param nullable: boolean, whether the field can be null (None) or not.
"""
def __init__(self, nullable=True):
super(SmallIntType, self).__init__(nullable)
class IntType(IntegralType):
"""
Int data types. SQL INT (32bits)
:param nullable: boolean, whether the field can be null (None) or not.
"""
def __init__(self, nullable=True):
super(IntType, self).__init__(nullable)
class BigIntType(IntegralType):
"""
Long data types. SQL BIGINT (64bits)
:param nullable: boolean, whether the field can be null (None) or not.
"""
def __init__(self, nullable=True):
super(BigIntType, self).__init__(nullable)
class FloatType(FractionalType):
"""
Float data type. SQL FLOAT
:param nullable: boolean, whether the field can be null (None) or not.
"""
def __init__(self, nullable=True):
super(FloatType, self).__init__(nullable)
class DoubleType(FractionalType):
"""
Double data type. SQL DOUBLE
:param nullable: boolean, whether the field can be null (None) or not.
"""
def __init__(self, nullable=True):
super(DoubleType, self).__init__(nullable)
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 create a DecimalType, the default precision and scale is (10, 0). When infer
schema from decimal.Decimal objects, it will be DecimalType(38, 18).
:param precision: the number of digits in a number (default: 10)
:param scale: the number of digits on right side of dot. (default: 0)
:param nullable: boolean, whether the field can be null (None) or not.
"""
def __init__(self, precision=10, scale=0, nullable=True):
super(DecimalType, self).__init__(nullable)
assert 1 <= precision <= 38
assert 0 <= scale <= precision
self.precision = precision
self.scale = scale
self.has_precision_info = True # this is public API
def __repr__(self):
return "DecimalType(%d, %d, %s)" % (self.precision, self.scale, str(self._nullable).lower())
class DateType(AtomicType):
"""
Date data type. SQL DATE
:param nullable: boolean, whether the field can be null (None) or not.
"""
def __init__(self, nullable=True):
super(DateType, self).__init__(nullable)
EPOCH_ORDINAL = datetime.datetime(1970, 1, 1).toordinal()
def need_conversion(self):
return True
def to_sql_type(self, d):
if d is not None:
return d.toordinal() - self.EPOCH_ORDINAL
def from_sql_type(self, v):
if v is not None:
return datetime.date.fromordinal(v + self.EPOCH_ORDINAL)
class TimeType(AtomicType):
"""
Time data type. SQL TIME
The precision must be greater than or equal to 0 and less than or equal to 9.
:param precision: int, the number of digits of fractional seconds (default: 0)
:param nullable: boolean, whether the field can be null (None) or not.
"""
EPOCH_ORDINAL = calendar.timegm(time.localtime(0)) * 10 ** 6
def __init__(self, precision=0, nullable=True):
super(TimeType, self).__init__(nullable)
assert 0 <= precision <= 9
self.precision = precision
def __repr__(self):
return "TimeType(%s, %s)" % (self.precision, str(self._nullable).lower())
def need_conversion(self):
return True
def to_sql_type(self, t):
if t is not None:
if t.tzinfo is not None:
offset = t.utcoffset()
offset = offset if offset else datetime.timedelta()
offset_microseconds =\
(offset.days * 86400 + offset.seconds) * 10 ** 6 + offset.microseconds
else:
offset_microseconds = self.EPOCH_ORDINAL
minutes = t.hour * 60 + t.minute
seconds = minutes * 60 + t.second
return seconds * 10 ** 6 + t.microsecond - offset_microseconds
def from_sql_type(self, t):
if t is not None:
seconds, microseconds = divmod(t + self.EPOCH_ORDINAL, 10 ** 6)
minutes, seconds = divmod(seconds, 60)
hours, minutes = divmod(minutes, 60)
return datetime.time(hours, minutes, seconds, microseconds)
class TimestampType(AtomicType):
"""
Timestamp data type. SQL TIMESTAMP WITHOUT TIME ZONE.
Consisting of ``year-month-day hour:minute:second[.fractional]`` with up to nanosecond
precision and values ranging from ``0000-01-01 00:00:00.000000000`` to
``9999-12-31 23:59:59.999999999``. Compared to the SQL standard, leap seconds (23:59:60 and
23:59:61) are not supported.
This class does not store or represent a time-zone. Instead, it is a description of
the date, as used for birthdays, combined with the local time as seen on a wall clock.
It cannot represent an instant on the time-line without additional information
such as an offset or time-zone.
The precision must be greater than or equal to 0 and less than or equal to 9.
:param precision: int, the number of digits of fractional seconds (default: 6)
:param nullable: boolean, whether the field can be null (None) or not.
"""
def __init__(self, precision=6, nullable=True):
super(TimestampType, self).__init__(nullable)
assert 0 <= precision <= 9
self.precision = precision
def __repr__(self):
return "TimestampType(%s, %s)" % (self.precision, str(self._nullable).lower())
def need_conversion(self):
return True
def to_sql_type(self, dt):
if dt is not None:
seconds = (calendar.timegm(dt.utctimetuple()) if dt.tzinfo
else time.mktime(dt.timetuple()))
return int(seconds) * 10 ** 6 + dt.microsecond
def from_sql_type(self, ts):
if ts is not None:
return datetime.datetime.fromtimestamp(ts // 10 ** 6).replace(microsecond=ts % 10 ** 6)
class LocalZonedTimestampType(AtomicType):
"""
Timestamp data type. SQL TIMESTAMP WITH LOCAL TIME ZONE.
Consisting of ``year-month-day hour:minute:second[.fractional] zone`` with up to nanosecond
precision and values ranging from ``0000-01-01 00:00:00.000000000 +14:59`` to
``9999-12-31 23:59:59.999999999 -14:59``. Compared to the SQL standard, Leap seconds (23:59:60
and 23:59:61) are not supported.
The value will be stored internally as a long value which stores all date and time
fields, to a precision of nanoseconds, as well as the offset from UTC/Greenwich.
The precision must be greater than or equal to 0 and less than or equal to 9.
:param precision: int, the number of digits of fractional seconds (default: 6)
:param nullable: boolean, whether the field can be null (None) or not.
"""
EPOCH_ORDINAL = calendar.timegm(time.localtime(0)) * 10 ** 6
def __init__(self, precision=6, nullable=True):
super(LocalZonedTimestampType, self).__init__(nullable)
assert 0 <= precision <= 9
self.precision = precision
def __repr__(self):
return "LocalZonedTimestampType(%s, %s)" % (self.precision, str(self._nullable).lower())
def need_conversion(self):
return True
def to_sql_type(self, dt):
if dt is not None:
seconds = (calendar.timegm(dt.utctimetuple()) if dt.tzinfo
else time.mktime(dt.timetuple()))
return int(seconds) * 10 ** 6 + dt.microsecond + self.EPOCH_ORDINAL
def from_sql_type(self, ts):
if ts is not None:
ts = ts - self.EPOCH_ORDINAL
return datetime.datetime.fromtimestamp(ts // 10 ** 6).replace(microsecond=ts % 10 ** 6)
class ZonedTimestampType(AtomicType):
"""
Timestamp data type with time zone. SQL TIMESTAMP WITH TIME ZONE.
Consisting of ``year-month-day hour:minute:second[.fractional] zone`` with up to nanosecond
precision and values ranging from {@code 0000-01-01 00:00:00.000000000 +14:59} to
``9999-12-31 23:59:59.999999999 -14:59``. Compared to the SQL standard, leap seconds (23:59:60
and 23:59:61) are not supported.
The value will be stored internally all date and time fields, to a precision of
nanoseconds, and a time-zone, with a zone offset used to handle ambiguous local date-times.
The precision must be greater than or equal to 0 and less than or equal to 9.
:param precision: int, the number of digits of fractional seconds (default: 6)
:param nullable: boolean, whether the field can be null (None) or not.
"""
def __init__(self, precision=6, nullable=True):
super(ZonedTimestampType, self).__init__(nullable)
assert 0 <= precision <= 9
self.precision = precision
def __repr__(self):
return "ZonedTimestampType(%s, %s)" % (self.precision, str(self._nullable).lower())
def need_conversion(self):
return True
def to_sql_type(self, dt):
if dt is not None:
seconds = (calendar.timegm(dt.utctimetuple()) if dt.tzinfo
else time.mktime(dt.timetuple()))
tzinfo = dt.tzinfo if dt.tzinfo else datetime.datetime.now(
datetime.timezone.utc).astimezone().tzinfo
offset = int(tzinfo.utcoffset(dt).total_seconds())
return int(seconds + offset) * 10 ** 6 + dt.microsecond, offset
def from_sql_type(self, zoned_ts):
if zoned_ts is not None:
from dateutil import tz
ts = zoned_ts[0] - zoned_ts[1] * 10 ** 6
tzinfo = tz.tzoffset(None, zoned_ts[1])
return datetime.datetime.fromtimestamp(ts // 10 ** 6, tz=tzinfo).replace(
microsecond=ts % 10 ** 6)
class Resolution(object):
"""
Helper class for defining the resolution of an interval.
:param unit: value defined in the constants of :class:`IntervalUnit`.
:param precision: the number of digits of years (=year precision) or the number of digits of
days (=day precision) or the number of digits of fractional seconds (
=fractional precision).
"""
class IntervalUnit(Enum):
SECOND = 0
MINUTE = 1
HOUR = 2
DAY = 3
MONTH = 4
YEAR = 5
def __init__(self, unit, precision=-1):
self._unit = unit
self._precision = precision
@property
def unit(self):
return self._unit
@property
def precision(self):
return self._precision
def __str__(self):
return '%s(%s)' % (str(self._unit), str(self._precision))
class YearMonthIntervalType(AtomicType):
"""
Year-month interval types. The type must be parameterized to one of the following
resolutions: interval of years, interval of years to months, or interval of months.
An interval of year-month consists of ``+years-months`` with values ranging from ``-9999-11``
to ``+9999-11``. The value representation is the same for all types of resolutions. For
example, an interval of months of 50 is always represented in an interval-of-years-to-months
format (with default year precision): ``+04-02``.
:param resolution: value defined in the constants of :class:`YearMonthResolution`,
representing one of the following resolutions: interval of years,
interval of years to months, or interval of months.
:param precision: int, the number of digits of years, must have a value
between 1 and 4 (both inclusive), default (2).
:param nullable: boolean, whether the field can be null (None) or not.
"""
class YearMonthResolution(object):
"""
Supported resolutions of :class:`YearMonthIntervalType`.
"""
YEAR = 1
MONTH = 2
YEAR_TO_MONTH = 3
DEFAULT_PRECISION = 2
def __init__(self, resolution, precision=DEFAULT_PRECISION, nullable=True):
assert resolution == YearMonthIntervalType.YearMonthResolution.YEAR or \
resolution == YearMonthIntervalType.YearMonthResolution.MONTH or \
resolution == YearMonthIntervalType.YearMonthResolution.YEAR_TO_MONTH
assert resolution != YearMonthIntervalType.YearMonthResolution.MONTH or \
precision == self.DEFAULT_PRECISION
assert 1 <= precision <= 4
self._resolution = resolution
self._precision = precision
super(YearMonthIntervalType, self).__init__(nullable)
@property
def resolution(self):
return self._resolution
@property
def precision(self):
return self._precision
class DayTimeIntervalType(AtomicType):
"""
Day-time interval types. The type must be parameterized to one of the following resolutions
with up to nanosecond precision: interval of days, interval of days to hours, interval of
days to minutes, interval of days to seconds, interval of hours, interval of hours to minutes,
interval of hours to seconds, interval of minutes, interval of minutes to seconds,
or interval of seconds.
An interval of day-time consists of ``+days hours:months:seconds.fractional`` with values
ranging from ``-999999 23:59:59.999999999`` to ``+999999 23:59:59.999999999``. The value
representation is the same for all types of resolutions. For example, an interval of seconds
of 70 is always represented in an interval-of-days-to-seconds format (with default precisions):
``+00 00:01:10.000000``.
:param resolution: value defined in the constants of :class:`DayTimeResolution`,
representing one of the following resolutions: interval of days, interval
of days to hours, interval of days to minutes, interval of days to seconds,
interval of hours, interval of hours to minutes, interval of hours to
seconds, interval of minutes, interval of minutes to seconds, or interval
of seconds.
:param day_precision: the number of digits of days, must have a value between 1 and 6 (both
inclusive) (default 2).
:param fractional_precision: the number of digits of fractional seconds, must have a value
between 0 and 9 (both inclusive) (default 6).
"""
class DayTimeResolution(Enum):
"""
Supported resolutions of :class:`DayTimeIntervalType`.
"""
DAY = 1
DAY_TO_HOUR = 2
DAY_TO_MINUTE = 3
DAY_TO_SECOND = 4
HOUR = 5
HOUR_TO_MINUTE = 6
HOUR_TO_SECOND = 7
MINUTE = 8
MINUTE_TO_SECOND = 9
SECOND = 10
DEFAULT_DAY_PRECISION = 2
DEFAULT_FRACTIONAL_PRECISION = 6
def __init__(self, resolution, day_precision=DEFAULT_DAY_PRECISION,
fractional_precision=DEFAULT_FRACTIONAL_PRECISION, nullable=True):
assert resolution == DayTimeIntervalType.DayTimeResolution.DAY or \
resolution == DayTimeIntervalType.DayTimeResolution.DAY_TO_HOUR or \
resolution == DayTimeIntervalType.DayTimeResolution.DAY_TO_MINUTE or \
resolution == DayTimeIntervalType.DayTimeResolution.DAY_TO_SECOND or \
resolution == DayTimeIntervalType.DayTimeResolution.HOUR or \
resolution == DayTimeIntervalType.DayTimeResolution.HOUR_TO_MINUTE or \
resolution == DayTimeIntervalType.DayTimeResolution.HOUR_TO_SECOND or \
resolution == DayTimeIntervalType.DayTimeResolution.MINUTE or \
resolution == DayTimeIntervalType.DayTimeResolution.MINUTE_TO_SECOND or \
resolution == DayTimeIntervalType.DayTimeResolution.SECOND
assert not self._needs_default_day_precision(
resolution) or day_precision == self.DEFAULT_DAY_PRECISION
assert not self._needs_default_fractional_precision(
resolution) or fractional_precision == self.DEFAULT_FRACTIONAL_PRECISION
assert 1 <= day_precision <= 6
assert 0 <= fractional_precision <= 9
self._resolution = resolution
self._day_precision = day_precision
self._fractional_precision = fractional_precision
super(DayTimeIntervalType, self).__init__(nullable)
def need_conversion(self):
return True
def to_sql_type(self, timedelta):
if timedelta is not None:
return (timedelta.days * 86400 + timedelta.seconds) * 10 ** 6 + timedelta.microseconds
def from_sql_type(self, ts):
if ts is not None:
return datetime.timedelta(microseconds=ts)
@property
def resolution(self) -> 'DayTimeIntervalType.DayTimeResolution':
return self._resolution
@property
def day_precision(self) -> int:
return self._day_precision
@property
def fractional_precision(self) -> int:
return self._fractional_precision
@staticmethod
def _needs_default_day_precision(resolution) -> bool:
if resolution == DayTimeIntervalType.DayTimeResolution.HOUR or \
resolution == DayTimeIntervalType.DayTimeResolution.HOUR_TO_MINUTE or \
resolution == DayTimeIntervalType.DayTimeResolution.HOUR_TO_SECOND or \
resolution == DayTimeIntervalType.DayTimeResolution.MINUTE or \
resolution == DayTimeIntervalType.DayTimeResolution.MINUTE_TO_SECOND or \
resolution == DayTimeIntervalType.DayTimeResolution.SECOND:
return True
else:
return False
@staticmethod
def _needs_default_fractional_precision(resolution) -> bool:
if resolution == DayTimeIntervalType.DayTimeResolution.DAY or \
resolution == DayTimeIntervalType.DayTimeResolution.DAY_TO_HOUR or \
resolution == DayTimeIntervalType.DayTimeResolution.DAY_TO_MINUTE or \
resolution == DayTimeIntervalType.DayTimeResolution.HOUR or \
resolution == DayTimeIntervalType.DayTimeResolution.HOUR_TO_MINUTE or \
resolution == DayTimeIntervalType.DayTimeResolution.MINUTE:
return True
else:
return False
_resolution_mappings = {
(Resolution.IntervalUnit.YEAR, None):
lambda p1, p2: YearMonthIntervalType(
YearMonthIntervalType.YearMonthResolution.YEAR, p1),
(Resolution.IntervalUnit.MONTH, None):
lambda p1, p2: YearMonthIntervalType(
YearMonthIntervalType.YearMonthResolution.MONTH),
(Resolution.IntervalUnit.YEAR, Resolution.IntervalUnit.MONTH):
lambda p1, p2: YearMonthIntervalType(
YearMonthIntervalType.YearMonthResolution.YEAR_TO_MONTH),
(Resolution.IntervalUnit.DAY, None):
lambda p1, p2: DayTimeIntervalType(
DayTimeIntervalType.DayTimeResolution.DAY,
p1,
DayTimeIntervalType.DEFAULT_FRACTIONAL_PRECISION),
(Resolution.IntervalUnit.DAY, Resolution.IntervalUnit.HOUR):
lambda p1, p2: DayTimeIntervalType(
DayTimeIntervalType.DayTimeResolution.DAY_TO_HOUR,
p1,
DayTimeIntervalType.DEFAULT_FRACTIONAL_PRECISION),
(Resolution.IntervalUnit.DAY, Resolution.IntervalUnit.MINUTE):
lambda p1, p2: DayTimeIntervalType(
DayTimeIntervalType.DayTimeResolution.DAY_TO_MINUTE,
p1,
DayTimeIntervalType.DEFAULT_FRACTIONAL_PRECISION),
(Resolution.IntervalUnit.DAY, Resolution.IntervalUnit.SECOND):
lambda p1, p2: DayTimeIntervalType(
DayTimeIntervalType.DayTimeResolution.DAY_TO_SECOND, p1, p2),
(Resolution.IntervalUnit.HOUR, None):
lambda p1, p2: DayTimeIntervalType(
DayTimeIntervalType.DayTimeResolution.HOUR),
(Resolution.IntervalUnit.HOUR, Resolution.IntervalUnit.MINUTE):
lambda p1, p2: DayTimeIntervalType(
DayTimeIntervalType.DayTimeResolution.HOUR_TO_MINUTE),
(Resolution.IntervalUnit.HOUR, Resolution.IntervalUnit.SECOND):
lambda p1, p2: DayTimeIntervalType(
DayTimeIntervalType.DayTimeResolution.HOUR_TO_SECOND,
DayTimeIntervalType.DEFAULT_DAY_PRECISION,
p2),
(Resolution.IntervalUnit.MINUTE, None):
lambda p1, p2: DayTimeIntervalType(
DayTimeIntervalType.DayTimeResolution.MINUTE),
(Resolution.IntervalUnit.MINUTE, Resolution.IntervalUnit.SECOND):
lambda p1, p2: DayTimeIntervalType(
DayTimeIntervalType.DayTimeResolution.MINUTE_TO_SECOND,
DayTimeIntervalType.DEFAULT_DAY_PRECISION,
p2),
(Resolution.IntervalUnit.SECOND, None):
lambda p1, p2: DayTimeIntervalType(
DayTimeIntervalType.DayTimeResolution.SECOND,
DayTimeIntervalType.DEFAULT_DAY_PRECISION,
p1)
}
def _from_resolution(upper_resolution: Resolution, lower_resolution: Resolution = None):
"""
Creates an interval type (YearMonthIntervalType or DayTimeIntervalType) from the
upper_resolution and lower_resolution.
"""
lower_unit = None if lower_resolution is None else lower_resolution.unit
lower_precision = -1 if lower_resolution is None else lower_resolution.precision
interval_type_provider = _resolution_mappings[(upper_resolution.unit, lower_unit)]
if interval_type_provider is None:
raise ValueError(
"Unsupported interval definition '%s TO %s'. Please check the documentation for "
"supported combinations for year-month and day-time intervals."
% (upper_resolution, lower_resolution))
return interval_type_provider(upper_resolution.precision, lower_precision)
def _from_java_interval_type(j_interval_type):
"""
Creates an interval type from the specified Java interval type.
:param j_interval_type: the Java interval type.
:return: :class:`YearMonthIntervalType` or :class:`DayTimeIntervalType`.
"""
gateway = get_gateway()
if is_instance_of(j_interval_type, gateway.jvm.YearMonthIntervalType):
resolution = j_interval_type.getResolution()
precision = j_interval_type.getYearPrecision()
def _from_java_year_month_resolution(j_resolution):
if j_resolution == gateway.jvm.YearMonthIntervalType.YearMonthResolution.YEAR:
return YearMonthIntervalType.YearMonthResolution.YEAR
elif j_resolution == gateway.jvm.YearMonthIntervalType.YearMonthResolution.MONTH:
return YearMonthIntervalType.YearMonthResolution.MONTH
else:
return YearMonthIntervalType.YearMonthResolution.YEAR_TO_MONTH
return YearMonthIntervalType(_from_java_year_month_resolution(resolution), precision)
else:
resolution = j_interval_type.getResolution()
day_precision = j_interval_type.getDayPrecision()
fractional_precision = j_interval_type.getFractionalPrecision()
def _from_java_day_time_resolution(j_resolution):
if j_resolution == gateway.jvm.DayTimeIntervalType.DayTimeResolution.DAY:
return DayTimeIntervalType.DayTimeResolution.DAY
elif j_resolution == gateway.jvm.DayTimeIntervalType.DayTimeResolution.DAY_TO_HOUR:
return DayTimeIntervalType.DayTimeResolution.DAY_TO_HOUR
elif j_resolution == gateway.jvm.DayTimeIntervalType.DayTimeResolution.DAY_TO_MINUTE:
return DayTimeIntervalType.DayTimeResolution.DAY_TO_MINUTE
elif j_resolution == gateway.jvm.DayTimeIntervalType.DayTimeResolution.DAY_TO_SECOND:
return DayTimeIntervalType.DayTimeResolution.DAY_TO_SECOND
elif j_resolution == gateway.jvm.DayTimeIntervalType.DayTimeResolution.HOUR:
return DayTimeIntervalType.DayTimeResolution.HOUR
elif j_resolution == gateway.jvm.DayTimeIntervalType.DayTimeResolution.HOUR_TO_MINUTE:
return DayTimeIntervalType.DayTimeResolution.HOUR_TO_MINUTE
elif j_resolution == gateway.jvm.DayTimeIntervalType.DayTimeResolution.HOUR_TO_SECOND:
return DayTimeIntervalType.DayTimeResolution.HOUR_TO_SECOND
elif j_resolution == gateway.jvm.DayTimeIntervalType.DayTimeResolution.MINUTE:
return DayTimeIntervalType.DayTimeResolution.MINUTE
elif j_resolution == gateway.jvm.DayTimeIntervalType.DayTimeResolution.MINUTE_TO_SECOND:
return DayTimeIntervalType.DayTimeResolution.MINUTE_TO_SECOND
else:
return DayTimeIntervalType.DayTimeResolution.SECOND
return DayTimeIntervalType(
_from_java_day_time_resolution(resolution), day_precision, fractional_precision)
_boxed_to_primitive_array_map = \
{'java.lang.Integer': '[I',
'java.lang.Long': '[J',
'java.lang.Byte': '[B',
'java.lang.Short': '[S',
'java.lang.Character': '[C',
'java.lang.Boolean': '[Z',
'java.lang.Float': '[F',
'java.lang.Double': '[D'}
class ArrayType(DataType):
"""
Array data type.
:param element_type: :class:`DataType` of each element in the array.
:param nullable: boolean, whether the field can be null (None) or not.
"""
def __init__(self, element_type, nullable=True):
"""
>>> ArrayType(VarCharType(100)) == ArrayType(VarCharType(100))
True
>>> ArrayType(VarCharType(100)) == ArrayType(BigIntType())
False
"""
assert isinstance(element_type, DataType), \
"element_type %s should be an instance of %s" % (element_type, DataType)
super(ArrayType, self).__init__(nullable)
self.element_type = element_type
def __repr__(self):
return "ArrayType(%s, %s)" % (repr(self.element_type), str(self._nullable).lower())
def need_conversion(self):
return self.element_type.need_conversion()
def to_sql_type(self, obj):
if not self.need_conversion():
return obj
return obj and [self.element_type.to_sql_type(v) for v in obj]
def from_sql_type(self, obj):
if not self.need_conversion():
return obj
return obj and [self.element_type.to_sql_type(v) for v in obj]
class ListViewType(DataType):
def __init__(self, element_type):
assert isinstance(element_type, DataType), \
"element_type %s should be an instance of %s" % (element_type, DataType)
super(ListViewType, self).__init__(False)
self._element_type = element_type
def __repr__(self):
return "ListViewType(%s)" % repr(self._element_type)
def to_sql_type(self, obj):
raise Exception("ListViewType can only be used in accumulator type declaration of "
"AggregateFunction.")
def from_sql_type(self, obj):
raise Exception("ListViewType can only be used in accumulator type declaration of "
"AggregateFunction.")
class MapType(DataType):
"""
Map data type.
:param key_type: :class:`DataType` of the keys in the map.
:param value_type: :class:`DataType` of the values in the map.
:param nullable: boolean, whether the field can be null (None) or not.
Keys in a map data type are not allowed to be null (None).
"""
def __init__(self, key_type, value_type, nullable=True):
"""
>>> (MapType(VarCharType(100, nullable=False), IntType())
... == MapType(VarCharType(100, nullable=False), IntType()))
True
>>> (MapType(VarCharType(100, nullable=False), IntType())
... == MapType(VarCharType(100, nullable=False), FloatType()))
False
"""
assert isinstance(key_type, DataType), \
"key_type %s should be an instance of %s" % (key_type, DataType)
assert isinstance(value_type, DataType), \
"value_type %s should be an instance of %s" % (value_type, DataType)
super(MapType, self).__init__(nullable)
self.key_type = key_type
self.value_type = value_type
def __repr__(self):
return "MapType(%s, %s, %s)" % (
repr(self.key_type), repr(self.value_type), str(self._nullable).lower())
def need_conversion(self):
return self.key_type.need_conversion() or self.value_type.need_conversion()
def to_sql_type(self, obj):
if not self.need_conversion():
return obj
return obj and dict((self.key_type.to_sql_type(k), self.value_type.to_sql_type(v))
for k, v in obj.items())
def from_sql_type(self, obj):
if not self.need_conversion():
return obj
return obj and dict((self.key_type.from_sql_type(k), self.value_type.from_sql_type(v))
for k, v in obj.items())
class MapViewType(DataType):
def __init__(self, key_type, value_type):
assert isinstance(key_type, DataType), \
"element_type %s should be an instance of %s" % (key_type, DataType)
assert isinstance(value_type, DataType), \