forked from databricks/koalas
-
Notifications
You must be signed in to change notification settings - Fork 0
/
base.py
762 lines (605 loc) · 24.2 KB
/
base.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
#
# Copyright (C) 2019 Databricks, Inc.
#
# Licensed 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.
#
"""
Base and utility classes for Koalas objects.
"""
from functools import wraps
from typing import Union
import numpy as np
import pandas as pd
from pandas.api.types import is_list_like
from pyspark import sql as spark
from pyspark.sql import functions as F, Window
from pyspark.sql.types import DoubleType, FloatType, LongType, StringType, TimestampType
from pyspark.sql.functions import monotonically_increasing_id
from databricks import koalas as ks # For running doctests and reference resolution in PyCharm.
from databricks.koalas.internal import _InternalFrame
from databricks.koalas.typedef import pandas_wraps, spark_type_to_pandas_dtype
from databricks.koalas.utils import align_diff_series, scol_for
def _column_op(f):
"""
A decorator that wraps APIs taking/returning Spark Column so that Koalas Series can be
supported too. If this decorator is used for the `f` function that takes Spark Column and
returns Spark Column, decorated `f` takes Koalas Series as well and returns Koalas
Series.
:param f: a function that takes Spark Column and returns Spark Column.
:param self: Koalas Series
:param args: arguments that the function `f` takes.
"""
@wraps(f)
def wrapper(self, *args):
# It is possible for the function `f` takes other arguments than Spark Column.
# To cover this case, explicitly check if the argument is Koalas Series and
# extract Spark Column. For other arguments, they are used as are.
cols = [arg for arg in args if isinstance(arg, IndexOpsMixin)]
if all(self._kdf is col._kdf for col in cols):
# Same DataFrame anchors
args = [arg._scol if isinstance(arg, IndexOpsMixin) else arg for arg in args]
scol = f(self._scol, *args)
# check if f is a logistic operator
log_ops = ['eq', 'ne', 'lt', 'le', 'ge', 'gt']
is_log_op = any(f == getattr(spark.Column, '__{}__'.format(log_op))
for log_op in log_ops)
if is_log_op:
filler = f == spark.Column.__ne__
scol = F.when(scol.isNull(), filler).otherwise(scol)
elif f == spark.Column.__or__:
scol = F.when(self._scol.isNull() | scol.isNull(), False).otherwise(scol)
elif f == spark.Column.__and__:
scol = F.when(scol.isNull(), False).otherwise(scol)
return self._with_new_scol(scol)
else:
# Different DataFrame anchors
def apply_func(this_column, *that_columns):
return f(this_column, *that_columns)
return align_diff_series(apply_func, self, *args, how="full")
return wrapper
def _numpy_column_op(f):
@wraps(f)
def wrapper(self, *args):
# PySpark does not support NumPy type out of the box. For now, we convert NumPy types
# into some primitive types understandable in PySpark.
new_args = []
for arg in args:
# TODO: This is a quick hack to support NumPy type. We should revisit this.
if isinstance(self.spark_type, LongType) and isinstance(arg, np.timedelta64):
new_args.append(float(arg / np.timedelta64(1, 's')))
else:
new_args.append(arg)
return _column_op(f)(self, *new_args)
return wrapper
def _wrap_accessor_spark(accessor, fn, return_type=None):
"""
Wrap an accessor property or method, e.g., Series.dt.date with a spark function.
"""
if return_type:
return _column_op(
lambda col: fn(col).cast(return_type)
)(accessor._data)
else:
return _column_op(fn)(accessor._data)
def _wrap_accessor_pandas(accessor, fn, return_type):
"""
Wrap an accessor property or method, e.g, Series.dt.date with a pandas function.
"""
return pandas_wraps(fn, return_col=return_type)(accessor._data)
class IndexOpsMixin(object):
"""common ops mixin to support a unified interface / docs for Series / Index
Assuming there are following attributes or properties and function.
:ivar _scol: Spark Column instance
:type _scol: pyspark.Column
:ivar _kdf: Parent's Koalas DataFrame
:type _kdf: ks.DataFrame
:ivar spark_type: Spark data type
:type spark_type: spark.types.DataType
def _with_new_scol(self, scol: spark.Column) -> IndexOpsMixin
Creates new object with the new column
"""
def __init__(self, internal: _InternalFrame, kdf):
self._internal = internal # type: _InternalFrame
self._kdf = kdf
@property
def _scol(self):
return self._internal.scol
# arithmetic operators
__neg__ = _column_op(spark.Column.__neg__)
def __add__(self, other):
if isinstance(self.spark_type, StringType):
# Concatenate string columns
if isinstance(other, IndexOpsMixin) and isinstance(other.spark_type, StringType):
return _column_op(F.concat)(self, other)
# Handle df['col'] + 'literal'
elif isinstance(other, str):
return _column_op(F.concat)(self, F.lit(other))
else:
raise TypeError('string addition can only be applied to string series or literals.')
else:
return _column_op(spark.Column.__add__)(self, other)
def __sub__(self, other):
# Note that timestamp subtraction casts arguments to integer. This is to mimic Pandas's
# behaviors. Pandas returns 'timedelta64[ns]' from 'datetime64[ns]'s subtraction.
if isinstance(other, IndexOpsMixin) and isinstance(self.spark_type, TimestampType):
if not isinstance(other.spark_type, TimestampType):
raise TypeError('datetime subtraction can only be applied to datetime series.')
return self.astype('bigint') - other.astype('bigint')
else:
return _column_op(spark.Column.__sub__)(self, other)
__mul__ = _column_op(spark.Column.__mul__)
__div__ = _numpy_column_op(spark.Column.__div__)
__truediv__ = _numpy_column_op(spark.Column.__truediv__)
__mod__ = _column_op(spark.Column.__mod__)
def __radd__(self, other):
# Handle 'literal' + df['col']
if isinstance(self.spark_type, StringType) and isinstance(other, str):
return self._with_new_scol(F.concat(F.lit(other), self._scol))
else:
return _column_op(spark.Column.__radd__)(self, other)
__rsub__ = _column_op(spark.Column.__rsub__)
__rmul__ = _column_op(spark.Column.__rmul__)
__rdiv__ = _numpy_column_op(spark.Column.__rdiv__)
__rtruediv__ = _numpy_column_op(spark.Column.__rtruediv__)
def __floordiv__(self, other):
return self._with_new_scol(
F.floor(_numpy_column_op(spark.Column.__div__)(self, other)._scol))
def __rfloordiv__(self, other):
return self._with_new_scol(
F.floor(_numpy_column_op(spark.Column.__rdiv__)(self, other)._scol))
__rmod__ = _column_op(spark.Column.__rmod__)
__pow__ = _column_op(spark.Column.__pow__)
__rpow__ = _column_op(spark.Column.__rpow__)
# logistic operators
__eq__ = _column_op(spark.Column.__eq__)
__ne__ = _column_op(spark.Column.__ne__)
__lt__ = _column_op(spark.Column.__lt__)
__le__ = _column_op(spark.Column.__le__)
__ge__ = _column_op(spark.Column.__ge__)
__gt__ = _column_op(spark.Column.__gt__)
# `and`, `or`, `not` cannot be overloaded in Python,
# so use bitwise operators as boolean operators
__and__ = _column_op(spark.Column.__and__)
__or__ = _column_op(spark.Column.__or__)
__invert__ = _column_op(spark.Column.__invert__)
__rand__ = _column_op(spark.Column.__rand__)
__ror__ = _column_op(spark.Column.__ror__)
@property
def dtype(self):
"""Return the dtype object of the underlying data.
Examples
--------
>>> s = ks.Series([1, 2, 3])
>>> s.dtype
dtype('int64')
>>> s = ks.Series(list('abc'))
>>> s.dtype
dtype('O')
>>> s = ks.Series(pd.date_range('20130101', periods=3))
>>> s.dtype
dtype('<M8[ns]')
>>> s.rename("a").to_frame().set_index("a").index.dtype
dtype('<M8[ns]')
"""
return spark_type_to_pandas_dtype(self.spark_type)
@property
def empty(self):
"""
Returns true if the current object is empty. Otherwise, returns false.
>>> ks.range(10).id.empty
False
>>> ks.range(0).id.empty
True
>>> ks.DataFrame({}, index=list('abc')).index.empty
False
"""
return self._internal._sdf.rdd.isEmpty()
@property
def hasnans(self):
"""
Return True if it has any missing values. Otherwise, it returns False.
>>> ks.DataFrame({}, index=list('abc')).index.hasnans
False
>>> ks.Series(['a', None]).hasnans
True
>>> ks.Series([1.0, 2.0, np.nan]).hasnans
True
>>> ks.Series([1, 2, 3]).hasnans
False
>>> ks.Series([1, 2, 3]).rename("a").to_frame().set_index("a").index.hasnans
False
"""
sdf = self._internal._sdf.select(self._scol)
col = self._scol
ret = sdf.select(F.max(col.isNull() | F.isnan(col))).collect()[0][0]
return ret
@property
def is_monotonic(self):
"""
Return boolean if values in the object are monotonically increasing.
.. note:: the current implementation of is_monotonic_increasing uses Spark's
Window without specifying partition specification. This leads to move all data into
single partition in single machine and could cause serious
performance degradation. Avoid this method against very large dataset.
Returns
-------
is_monotonic : boolean
Examples
--------
>>> ser = ks.Series(['1/1/2018', '3/1/2018', '4/1/2018'])
>>> ser.is_monotonic
True
>>> df = ks.DataFrame({'dates': [None, '1/1/2018', '2/1/2018', '3/1/2018']})
>>> df.dates.is_monotonic
False
>>> df.index.is_monotonic
True
>>> ser = ks.Series([1])
>>> ser.is_monotonic
True
>>> ser = ks.Series([])
>>> ser.is_monotonic
True
>>> ser.rename("a").to_frame().set_index("a").index.is_monotonic
True
>>> ser = ks.Series([5, 4, 3, 2, 1], index=[1, 2, 3, 4, 5])
>>> ser.is_monotonic
False
>>> ser.index.is_monotonic
True
"""
col = self._scol
window = Window.orderBy(monotonically_increasing_id()).rowsBetween(-1, -1)
return self._with_new_scol((col >= F.lag(col, 1).over(window)) & col.isNotNull()).all()
is_monotonic_increasing = is_monotonic
@property
def is_monotonic_decreasing(self):
"""
Return boolean if values in the object are monotonically decreasing.
.. note:: the current implementation of is_monotonic_decreasing uses Spark's
Window without specifying partition specification. This leads to move all data into
single partition in single machine and could cause serious
performance degradation. Avoid this method against very large dataset.
Returns
-------
is_monotonic : boolean
Examples
--------
>>> ser = ks.Series(['4/1/2018', '3/1/2018', '1/1/2018'])
>>> ser.is_monotonic_decreasing
True
>>> df = ks.DataFrame({'dates': [None, '3/1/2018', '2/1/2018', '1/1/2018']})
>>> df.dates.is_monotonic_decreasing
False
>>> df.index.is_monotonic_decreasing
False
>>> ser = ks.Series([1])
>>> ser.is_monotonic_decreasing
True
>>> ser = ks.Series([])
>>> ser.is_monotonic_decreasing
True
>>> ser.rename("a").to_frame().set_index("a").index.is_monotonic_decreasing
True
>>> ser = ks.Series([5, 4, 3, 2, 1], index=[1, 2, 3, 4, 5])
>>> ser.is_monotonic_decreasing
True
>>> ser.index.is_monotonic_decreasing
False
"""
col = self._scol
window = Window.orderBy(monotonically_increasing_id()).rowsBetween(-1, -1)
return self._with_new_scol((col <= F.lag(col, 1).over(window)) & col.isNotNull()).all()
@property
def ndim(self):
"""
Return an int representing the number of array dimensions.
Return 1 for Series / Index / MultiIndex.
Examples
--------
For Series
>>> s = ks.Series([None, 1, 2, 3, 4], index=[4, 5, 2, 1, 8])
>>> s.ndim
1
For Index
>>> s.index.ndim
1
For MultiIndex
>>> midx = pd.MultiIndex([['lama', 'cow', 'falcon'],
... ['speed', 'weight', 'length']],
... [[0, 0, 0, 1, 1, 1, 2, 2, 2],
... [1, 1, 1, 1, 1, 2, 1, 2, 2]])
>>> s = ks.Series([45, 200, 1.2, 30, 250, 1.5, 320, 1, 0.3], index=midx)
>>> s.index.ndim
1
"""
return 1
def astype(self, dtype):
"""
Cast a Koalas object to a specified dtype ``dtype``.
Parameters
----------
dtype : data type
Use a numpy.dtype or Python type to cast entire pandas object to
the same type.
Returns
-------
casted : same type as caller
See Also
--------
to_datetime : Convert argument to datetime.
Examples
--------
>>> ser = ks.Series([1, 2], dtype='int32')
>>> ser
0 1
1 2
Name: 0, dtype: int32
>>> ser.astype('int64')
0 1
1 2
Name: 0, dtype: int64
>>> ser.rename("a").to_frame().set_index("a").index.astype('int64')
Int64Index([1, 2], dtype='int64', name='a')
"""
from databricks.koalas.typedef import as_spark_type
spark_type = as_spark_type(dtype)
if not spark_type:
raise ValueError("Type {} not understood".format(dtype))
return self._with_new_scol(self._scol.cast(spark_type))
def isin(self, values):
"""
Check whether `values` are contained in Series.
Return a boolean Series showing whether each element in the Series
matches an element in the passed sequence of `values` exactly.
Parameters
----------
values : list or set
The sequence of values to test.
Returns
-------
isin : Series (bool dtype)
Examples
--------
>>> s = ks.Series(['lama', 'cow', 'lama', 'beetle', 'lama',
... 'hippo'], name='animal')
>>> s.isin(['cow', 'lama'])
0 True
1 True
2 True
3 False
4 True
5 False
Name: animal, dtype: bool
Passing a single string as ``s.isin('lama')`` will raise an error. Use
a list of one element instead:
>>> s.isin(['lama'])
0 True
1 False
2 True
3 False
4 True
5 False
Name: animal, dtype: bool
>>> s.rename("a").to_frame().set_index("a").index.isin(['lama'])
Index([True, False, True, False, True, False], dtype='object', name='a')
"""
if not is_list_like(values):
raise TypeError("only list-like objects are allowed to be passed"
" to isin(), you passed a [{values_type}]"
.format(values_type=type(values).__name__))
return self._with_new_scol(self._scol.isin(list(values))).rename(self.name)
def isnull(self):
"""
Detect existing (non-missing) values.
Return a boolean same-sized object indicating if the values are NA.
NA values, such as None or numpy.NaN, gets mapped to True values.
Everything else gets mapped to False values. Characters such as empty strings '' or
numpy.inf are not considered NA values
(unless you set pandas.options.mode.use_inf_as_na = True).
Returns
-------
Series : Mask of bool values for each element in Series
that indicates whether an element is not an NA value.
Examples
--------
>>> ser = ks.Series([5, 6, np.NaN])
>>> ser.isna() # doctest: +NORMALIZE_WHITESPACE
0 False
1 False
2 True
Name: 0, dtype: bool
>>> ser.rename("a").to_frame().set_index("a").index.isna()
Index([False, False, True], dtype='object', name='a')
"""
if isinstance(self.spark_type, (FloatType, DoubleType)):
return self._with_new_scol(self._scol.isNull() | F.isnan(self._scol)).rename(self.name)
else:
return self._with_new_scol(self._scol.isNull()).rename(self.name)
isna = isnull
def notnull(self):
"""
Detect existing (non-missing) values.
Return a boolean same-sized object indicating if the values are not NA.
Non-missing values get mapped to True.
Characters such as empty strings '' or numpy.inf are not considered NA values
(unless you set pandas.options.mode.use_inf_as_na = True).
NA values, such as None or numpy.NaN, get mapped to False values.
Returns
-------
Series : Mask of bool values for each element in Series
that indicates whether an element is not an NA value.
Examples
--------
Show which entries in a Series are not NA.
>>> ser = ks.Series([5, 6, np.NaN])
>>> ser
0 5.0
1 6.0
2 NaN
Name: 0, dtype: float64
>>> ser.notna()
0 True
1 True
2 False
Name: 0, dtype: bool
>>> ser.rename("a").to_frame().set_index("a").index.notna()
Index([True, True, False], dtype='object', name='a')
"""
return (~self.isnull()).rename(self.name)
notna = notnull
# TODO: axis, skipna, and many arguments should be implemented.
def all(self, axis: Union[int, str] = 0) -> bool:
"""
Return whether all elements are True.
Returns True unless there at least one element within a series that is
False or equivalent (e.g. zero or empty)
Parameters
----------
axis : {0 or 'index'}, default 0
Indicate which axis or axes should be reduced.
* 0 / 'index' : reduce the index, return a Series whose index is the
original column labels.
Examples
--------
>>> ks.Series([True, True]).all()
True
>>> ks.Series([True, False]).all()
False
>>> ks.Series([0, 1]).all()
False
>>> ks.Series([1, 2, 3]).all()
True
>>> ks.Series([True, True, None]).all()
True
>>> ks.Series([True, False, None]).all()
False
>>> ks.Series([]).all()
True
>>> ks.Series([np.nan]).all()
True
>>> df = ks.Series([True, False, None]).rename("a").to_frame()
>>> df.set_index("a").index.all()
False
"""
if axis not in [0, 'index']:
raise ValueError('axis should be either 0 or "index" currently.')
sdf = self._internal._sdf.select(self._scol)
col = scol_for(sdf, sdf.columns[0])
# Note that we're ignoring `None`s here for now.
# any and every was added as of Spark 3.0
# ret = sdf.select(F.expr("every(CAST(`%s` AS BOOLEAN))" % sdf.columns[0])).collect()[0][0]
# Here we use min as its alternative:
ret = sdf.select(F.min(F.coalesce(col.cast('boolean'), F.lit(True)))).collect()[0][0]
if ret is None:
return True
else:
return ret
# TODO: axis, skipna, and many arguments should be implemented.
def any(self, axis: Union[int, str] = 0) -> bool:
"""
Return whether any element is True.
Returns False unless there at least one element within a series that is
True or equivalent (e.g. non-zero or non-empty).
Parameters
----------
axis : {0 or 'index'}, default 0
Indicate which axis or axes should be reduced.
* 0 / 'index' : reduce the index, return a Series whose index is the
original column labels.
Examples
--------
>>> ks.Series([False, False]).any()
False
>>> ks.Series([True, False]).any()
True
>>> ks.Series([0, 0]).any()
False
>>> ks.Series([0, 1, 2]).any()
True
>>> ks.Series([False, False, None]).any()
False
>>> ks.Series([True, False, None]).any()
True
>>> ks.Series([]).any()
False
>>> ks.Series([np.nan]).any()
False
>>> df = ks.Series([True, False, None]).rename("a").to_frame()
>>> df.set_index("a").index.any()
True
"""
if axis not in [0, 'index']:
raise ValueError('axis should be either 0 or "index" currently.')
sdf = self._internal._sdf.select(self._scol)
col = scol_for(sdf, sdf.columns[0])
# Note that we're ignoring `None`s here for now.
# any and every was added as of Spark 3.0
# ret = sdf.select(F.expr("any(CAST(`%s` AS BOOLEAN))" % sdf.columns[0])).collect()[0][0]
# Here we use max as its alternative:
ret = sdf.select(F.max(F.coalesce(col.cast('boolean'), F.lit(False)))).collect()[0][0]
if ret is None:
return False
else:
return ret
# TODO: add frep and axis parameter
def shift(self, periods=1, fill_value=None):
"""
Shift Series/Index by desired number of periods.
.. note:: the current implementation of shift uses Spark's Window without
specifying partition specification. This leads to move all data into
single partition in single machine and could cause serious
performance degradation. Avoid this method against very large dataset.
Parameters
----------
periods : int
Number of periods to shift. Can be positive or negative.
fill_value : object, optional
The scalar value to use for newly introduced missing values.
The default depends on the dtype of self. For numeric data, np.nan is used.
Returns
-------
Copy of input Series/Index, shifted.
Examples
--------
>>> df = ks.DataFrame({'Col1': [10, 20, 15, 30, 45],
... 'Col2': [13, 23, 18, 33, 48],
... 'Col3': [17, 27, 22, 37, 52]},
... columns=['Col1', 'Col2', 'Col3'])
>>> df.Col1.shift(periods=3)
0 NaN
1 NaN
2 NaN
3 10.0
4 20.0
Name: Col1, dtype: float64
>>> df.Col2.shift(periods=3, fill_value=0)
0 0
1 0
2 0
3 13
4 23
Name: Col2, dtype: int64
>>> df.index.shift(periods=3, fill_value=0)
Int64Index([0, 0, 0, 0, 1], dtype='int64')
"""
return self._shift(periods, fill_value)
def _shift(self, periods, fill_value, part_cols=()):
if not isinstance(periods, int):
raise ValueError('periods should be an int; however, got [%s]' % type(periods))
col = self._scol
window = Window.partitionBy(*part_cols).orderBy(self._internal.index_scols)\
.rowsBetween(-periods, -periods)
lag_col = F.lag(col, periods).over(window)
col = F.when(lag_col.isNull() | F.isnan(lag_col), fill_value).otherwise(lag_col)
return self._with_new_scol(col).rename(self.name)