-
-
Notifications
You must be signed in to change notification settings - Fork 46
/
core.py
932 lines (776 loc) · 31.2 KB
/
core.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
import numpy as np
import pandas as pd
import dask.dataframe as dd
import dask.array as da
from dask.dataframe.core import _emulate, map_partitions, elemwise, new_dd_object
from dask.highlevelgraph import HighLevelGraph
from dask.utils import M, OperatorMethodMixin, derived_from, ignore_warning
from dask.base import tokenize
import geopandas
from shapely.geometry.base import BaseGeometry
from shapely.geometry import box
import pygeos
from .hilbert_distance import _hilbert_distance
from .morton_distance import _morton_distance
from .geohash import _geohash
import dask_geopandas
def _set_crs(df, crs, allow_override):
"""Return a new object with crs set to ``crs``"""
return df.set_crs(crs, allow_override=allow_override)
class _Frame(dd.core._Frame, OperatorMethodMixin):
"""Superclass for DataFrame and Series
Parameters
----------
dsk : dict
The dask graph to compute this DataFrame
name : str
The key prefix that specifies which keys in the dask comprise this
particular DataFrame / Series
meta : geopandas.GeoDataFrame, geopandas.GeoSeries
An empty geopandas object with names, dtypes, and indices matching the
expected output.
divisions : tuple of index values
Values along which we partition our blocks on the index
"""
_partition_type = geopandas.base.GeoPandasBase
def __init__(self, dsk, name, meta, divisions, spatial_partitions=None):
super().__init__(dsk, name, meta, divisions)
self._spatial_partitions = spatial_partitions
def to_dask_dataframe(self):
"""Create a dask.dataframe object from a dask_geopandas object"""
return self.map_partitions(pd.DataFrame)
def __dask_postpersist__(self):
return self._rebuild, ()
def _rebuild(self, dsk, *, rename=None):
# this is a copy of the dask.dataframe version, only with the addition
# to pass self.spatial_partitions
name = self._name
if rename:
name = rename.get(name, name)
return type(self)(
dsk, name, self._meta, self.divisions, self.spatial_partitions
)
@property
def spatial_partitions(self):
"""
The spatial extent of each of the partitions of the dask GeoDataFrame.
"""
return self._spatial_partitions
@spatial_partitions.setter
def spatial_partitions(self, value):
if value is not None:
if not isinstance(value, geopandas.GeoSeries):
raise TypeError(
"Expected a geopandas.GeoSeries for the spatial_partitions, "
f"got {type(value)} instead."
)
if len(value) != self.npartitions:
raise ValueError(
f"Expected spatial partitions of length {self.npartitions}, "
f"got {len(value)} instead."
)
self._spatial_partitions = value
@classmethod
def _bind_property(cls, attr, preserve_spatial_partitions=False):
"""Map property to partitions and bind to class"""
def prop(self):
meta = getattr(self._meta, attr)
result = self.map_partitions(getattr, attr, token=attr, meta=meta)
if preserve_spatial_partitions:
result = self._propagate_spatial_partitions(result)
return result
doc = getattr(cls._partition_type, attr).__doc__
# Insert disclaimer that this is a copied docstring note that
# malformed docs will not get the disclaimer (see #4746).
if doc:
doc = ignore_warning(doc, cls._partition_type, attr)
setattr(cls, name, property(fget=prop, doc=doc))
@classmethod
def _bind_elemwise_comparison_method(
cls, name, comparison, original, *args, **kwargs
):
"""bind comparison method like GeoSeries.contains to this class"""
def meth(self, other, *args, **kwargs):
return elemwise(comparison, self, other, *args, **kwargs)
meth.__name__ = name
setattr(cls, name, derived_from(original)(meth))
@classmethod
def _bind_elemwise_operator_method(cls, name, op, original, *args, **kwargs):
"""bind operator method like GeoSeries.distance to this class"""
# name must be explicitly passed for div method whose name is truediv
def meth(self, other, *args, **kwargs):
meta = _emulate(op, self, other)
return map_partitions(
op, self, other, meta=meta, enforce_metadata=False, *args, **kwargs
)
meth.__name__ = name
setattr(cls, name, derived_from(original)(meth))
def calculate_spatial_partitions(self):
"""Calculate spatial partitions"""
# TEMP method to calculate spatial partitions for testing, need to
# add better methods (set_partitions / repartition)
parts = geopandas.GeoSeries(
self.map_partitions(
lambda part: pygeos.convex_hull(
pygeos.geometrycollections(part.geometry.values.data)
)
).compute(),
crs=self.crs,
)
self.spatial_partitions = parts
def _propagate_spatial_partitions(self, new_object):
"""
We need to override several dask methods to ensure the spatial
partitions are properly propagated.
This is a helper method to set this.
"""
new_object.spatial_partitions = self.spatial_partitions
return new_object
@property
@derived_from(geopandas.GeoSeries)
def crs(self):
"""
The Coordinate Reference System (CRS) represented as a ``pyproj.CRS``
object.
Returns None if the CRS is not set, and to set the value it
:getter: Returns a ``pyproj.CRS`` or None. When setting, the value
can be anything accepted by :meth:`pyproj.CRS.from_user_input`,
such as an authority string (eg "EPSG:4326") or a WKT string.
"""
return self._meta.crs
@crs.setter
def crs(self, value):
"""Sets the value of the crs"""
# When using setter, Geopandas always overrides the CRS
new = self.set_crs(value, allow_override=True)
self._meta = new._meta
self._name = new._name
self.dask = new.dask
@derived_from(geopandas.GeoSeries)
def set_crs(self, value, allow_override=False):
"""Set the value of the crs on a new object"""
new = self.map_partitions(
_set_crs, value, allow_override, enforce_metadata=False
)
if self.spatial_partitions is not None:
new.spatial_partitions = self.spatial_partitions.set_crs(
value, allow_override=allow_override
)
return new
@derived_from(geopandas.GeoSeries)
def to_crs(self, crs=None, epsg=None):
return self.map_partitions(M.to_crs, crs=crs, epsg=epsg)
def copy(self):
"""Make a copy of the dataframe
Creates shallow copies of the computational graph and spatial partitions.
Does not affect the underlying data.
"""
self_copy = super().copy()
if self.spatial_partitions is not None:
self_copy.spatial_partitions = self.spatial_partitions.copy()
return self_copy
@property
@derived_from(geopandas.base.GeoPandasBase)
def total_bounds(self):
def agg(concatted):
return np.array(
(
np.nanmin(concatted[0::4]), # minx
np.nanmin(concatted[1::4]), # miny
np.nanmax(concatted[2::4]), # maxx
np.nanmax(concatted[3::4]), # maxy
)
)
total_bounds = self.reduction(
lambda x: getattr(x, "total_bounds"),
token="total_bounds",
meta=self._meta.total_bounds,
aggregate=agg,
)
return da.Array(
total_bounds.dask,
total_bounds.name,
chunks=((4,),),
dtype=total_bounds.dtype,
)
@property
def sindex(self):
"""Need to figure out how to concatenate spatial indexes"""
raise NotImplementedError
@property
@derived_from(geopandas.base.GeoPandasBase)
def unary_union(self):
attr = "unary_union"
meta = BaseGeometry()
return self.reduction(
lambda x: getattr(x, attr),
token=attr,
aggregate=lambda x: getattr(geopandas.GeoSeries(x), attr),
meta=meta,
)
@derived_from(geopandas.base.GeoPandasBase)
def representative_point(self):
return self.map_partitions(
self._partition_type.representative_point, enforce_metadata=False
)
@derived_from(geopandas.base.GeoPandasBase)
def geom_equals_exact(self, other, tolerance):
comparison = self._partition_type.geom_equals_exact
return elemwise(comparison, self, other, tolerance)
@derived_from(geopandas.base.GeoPandasBase)
def buffer(self, distance, resolution=16, **kwargs):
return self.map_partitions(
self._partition_type.buffer,
distance,
resolution=resolution,
enforce_metadata=False,
**kwargs,
)
@derived_from(geopandas.base.GeoPandasBase)
def simplify(self, *args, **kwargs):
return self.map_partitions(
self._partition_type.simplify, *args, enforce_metadata=False, **kwargs
)
@derived_from(geopandas.base.GeoPandasBase)
def interpolate(self, distance, normalized=False):
return self.map_partitions(
self._partition_type.interpolate,
distance,
normalized=normalized,
enforce_metadata=False,
)
@derived_from(geopandas.base.GeoPandasBase)
def affine_transform(self, matrix):
return self.map_partitions(
self._partition_type.affine_transform, matrix, enforce_metadata=False
)
@derived_from(geopandas.base.GeoPandasBase)
def translate(self, xoff=0.0, yoff=0.0, zoff=0.0):
return self.map_partitions(
self._partition_type.translate,
xoff=xoff,
yoff=yoff,
zoff=zoff,
enforce_metadata=False,
)
@derived_from(geopandas.base.GeoPandasBase)
def rotate(self, angle, origin="center", use_radians=False):
return self.map_partitions(
self._partition_type.rotate,
angle,
origin=origin,
use_radians=use_radians,
enforce_metadata=False,
)
@derived_from(geopandas.base.GeoPandasBase)
def scale(self, xfact=1.0, yfact=1.0, zfact=1.0, origin="center"):
return self.map_partitions(
self._partition_type.scale,
xfact=xfact,
yfact=yfact,
zfact=zfact,
origin=origin,
enforce_metadata=False,
)
@derived_from(geopandas.base.GeoPandasBase)
def skew(self, xs=0.0, ys=0.0, origin="center", use_radians=False):
return self.map_partitions(
self._partition_type.skew,
xs=xs,
ys=ys,
origin=origin,
use_radians=use_radians,
enforce_metadata=False,
)
@derived_from(geopandas.geodataframe.GeoDataFrame)
def explode(self):
return self.map_partitions(self._partition_type.explode, enforce_metadata=False)
@property
@derived_from(geopandas.geodataframe.GeoDataFrame)
def cx(self):
"""
Coordinate based indexer to select by intersection with bounding box.
Format of input should be ``.cx[xmin:xmax, ymin:ymax]``. Any of
``xmin``, ``xmax``, ``ymin``, and ``ymax`` can be provided, but input
must include a comma separating x and y slices. That is, ``.cx[:, :]``
will return the full series/frame, but ``.cx[:]`` is not implemented.
"""
return _CoordinateIndexer(self)
def hilbert_distance(self, total_bounds=None, level=16):
"""
Calculate the distance along a Hilbert curve.
The distances are calculated for the midpoints of the geometries in the
GeoDataFrame, and using the total bounds of the GeoDataFrame.
The Hilbert distance can be used to spatially partition Dask-GeoPandas
objects, by mapping two dimensional geometries along the Hilbert curve.
Parameters
----------
total_bounds : 4-element array, optional
The spatial extent in which the curve is constructed (used to
rescale the geometry midpoints). By default, the total bounds
of the full dask GeoDataFrame will be computed (from the spatial
partitions, if available, otherwise computed from the full
dataframe). If known, you can pass the total bounds to avoid this
extra computation.
level : int (1 - 16), default 16
Determines the precision of the curve (points on the curve will
have coordinates in the range [0, 2^level - 1]).
Returns
-------
dask.Series
Series containing distances for each partition
"""
# Compute total bounds of all partitions rather than each partition
if total_bounds is None:
if self.spatial_partitions is not None:
total_bounds = self.spatial_partitions.total_bounds
else:
total_bounds = self.total_bounds
# Calculate hilbert distances for each partition
distances = self.map_partitions(
_hilbert_distance,
total_bounds=total_bounds,
level=level,
meta=pd.Series([], name="hilbert_distance", dtype="uint32"),
)
return distances
def morton_distance(self, total_bounds=None, level=16):
"""
Calculate the distance of geometries along the Morton curve
The Morton curve is also known as Z-order https://en.wikipedia.org/wiki/Z-order.
The Morton distance can be used to spatially partition Dask-GeoPandas objects,
by mapping two-dimensional geometries along the Morton space-filing curve.
Each geometry is represented by the midpoint of its bounds and linked to the
Morton curve. The function returns a distance from the beginning
of the curve to the linked point.
Morton distance is more performant than ``hilbert_distance`` but can result in
less optimal partitioning.
Parameters
----------
total_bounds : 4-element array, optional
The spatial extent in which the curve is constructed (used to
rescale the geometry midpoints). By default, the total bounds
of the full dask GeoDataFrame will be computed (from the spatial
partitions, if available, otherwise computed from the full
dataframe). If known, you can pass the total bounds to avoid this
extra computation.
level : int (1 - 16), default 16
Determines the precision of the Morton curve.
Returns
-------
dask.Series
Series containing distances along the Morton curve
"""
# Compute total bounds of all partitions rather than each partition
if total_bounds is None:
if self.spatial_partitions is not None:
total_bounds = self.spatial_partitions.total_bounds
else:
total_bounds = self.total_bounds
# Calculate Morton distances for each partition
distances = self.map_partitions(
_morton_distance,
total_bounds=total_bounds,
level=level,
meta=pd.Series([], name="morton_distance", dtype="uint32"),
)
return distances
def geohash(self, as_string=True, precision=12):
"""
Calculate geohash based on the middle points of the geometry bounds
for a given precision.
Only geographic coordinates (longitude, latitude) are supported.
Parameters
----------
as_string : bool, default True
To return string or int Geohash.
precision : int (1 - 12), default 12
Precision of the string geohash values. Only used when
``as_string=True``.
Returns
-------
type : pandas.Series
Series containing Geohash
"""
if precision not in range(1, 13):
raise ValueError(
"The Geohash precision only accepts an integer value between 1 and 12"
)
if as_string is True:
dtype = object
else:
dtype = np.uint64
geohashes = self.map_partitions(
_geohash,
as_string=as_string,
precision=precision,
meta=pd.Series([], name="geohash", dtype=dtype),
)
return geohashes
@derived_from(geopandas.GeoDataFrame)
def clip(self, mask, keep_geom_type=False):
return dask_geopandas.clip(self, mask=mask, keep_geom_type=keep_geom_type)
@derived_from(geopandas.GeoDataFrame)
def to_wkt(self, **kwargs):
meta = self._meta.to_wkt(**kwargs)
return self.map_partitions(M.to_wkt, **kwargs, meta=meta)
@derived_from(geopandas.GeoDataFrame)
def to_wkb(self, hex=False, **kwargs):
meta = self._meta.to_wkb(hex=hex, **kwargs)
return self.map_partitions(M.to_wkb, hex=hex, **kwargs, meta=meta)
class GeoSeries(_Frame, dd.core.Series):
"""Parallel GeoPandas GeoSeries
Do not use this class directly. Instead use functions like
:func:`dask_geopandas.read_parquet`,or :func:`dask_geopandas.from_geopandas`.
"""
_partition_type = geopandas.GeoSeries
class GeoDataFrame(_Frame, dd.core.DataFrame):
"""Parallel GeoPandas GeoDataFrame
Do not use this class directly. Instead use functions like
:func:`dask_geopandas.read_parquet`,or :func:`dask_geopandas.from_geopandas`.
"""
_partition_type = geopandas.GeoDataFrame
@property
def geometry(self):
geometry_column_name = self._meta._geometry_column_name
if geometry_column_name not in self.columns:
raise AttributeError(
"No geometry data set yet (expected in"
" column '%s'." % geometry_column_name
)
return self[geometry_column_name]
@geometry.setter
def geometry(self, col):
"""Sets the geometry column"""
new = self.set_geometry(col)
self._meta = new._meta
self._name = new._name
self.dask = new.dask
@derived_from(dd.DataFrame)
def set_index(self, *args, **kwargs):
"""Override to ensure we get GeoDataFrame with set geometry column"""
ddf = super().set_index(*args, **kwargs)
return ddf.set_geometry(self._meta.geometry.name)
@derived_from(geopandas.GeoDataFrame)
def set_geometry(self, col):
# calculate ourselves to use meta and not meta_nonempty, which would
# raise an error if meta is an invalid GeoDataFrame (e.g. geometry
# column name not yet set correctly)
if isinstance(col, GeoSeries):
meta = self._meta.set_geometry(col._meta)
else:
meta = self._meta.set_geometry(col)
return self.map_partitions(M.set_geometry, col, meta=meta)
@derived_from(geopandas.GeoDataFrame)
def rename_geometry(self, col):
meta = self._meta.rename_geometry(col)
return self.map_partitions(M.rename_geometry, col, meta=meta)
def __getitem__(self, key):
"""
If the result is a new dask_geopandas.GeoDataFrame/GeoSeries (automatically
determined by dask based on the meta), then pass through the spatial
partitions information.
"""
result = super().__getitem__(key)
if isinstance(result, _Frame):
result = self._propagate_spatial_partitions(result)
return result
def _repr_html_(self):
output = super()._repr_html_()
return output.replace(
"Dask DataFrame Structure", "Dask-GeoPandas GeoDataFrame Structure"
)
@derived_from(dd.DataFrame)
def to_parquet(self, path, *args, **kwargs):
"""See dask_geopadandas.to_parquet docstring for more information"""
from .io.parquet import to_parquet
return to_parquet(self, path, *args, **kwargs)
def to_feather(self, path, *args, **kwargs):
"""See dask_geopadandas.to_feather docstring for more information"""
from .io.arrow import to_feather
return to_feather(self, path, *args, **kwargs)
def dissolve(self, by=None, aggfunc="first", split_out=1, **kwargs):
"""Dissolve geometries within ``groupby`` into a single geometry.
Parameters
----------
by : string, default None
Column whose values define groups to be dissolved. If None,
whole GeoDataFrame is considered a single group.
aggfunc : function, string or dict, default "first"
Aggregation function for manipulation of data associated
with each group. Passed to dask ``groupby.agg`` method.
Note that ``aggfunc`` needs to be applicable to all columns (i.e. ``"mean"``
cannot be used with string dtype). Select only required columns before
``dissolve`` or pass a dictionary mapping to ``aggfunc`` to specify the
aggregation function for each column separately.
split_out : int, default 1
Number of partitions of the output
**kwargs
keyword arguments passed to ``groupby``
Examples
--------
>>> ddf.dissolve("foo", split_out=12)
>>> ddf[["foo", "bar", "geometry"]].dissolve("foo", aggfunc="mean")
>>> ddf.dissolve("foo", aggfunc={"bar": "mean", "baz": "first"})
"""
if by is None:
by = lambda x: 0
drop = [self.geometry.name]
else:
drop = [by, self.geometry.name]
def union(block):
merged_geom = block.unary_union
return merged_geom
merge_geometries = dd.Aggregation(
"merge_geometries", lambda s: s.agg(union), lambda s0: s0.agg(union)
)
if isinstance(aggfunc, dict):
data_agg = aggfunc
else:
data_agg = {col: aggfunc for col in self.columns.drop(drop)}
data_agg[self.geometry.name] = merge_geometries
aggregated = self.groupby(by=by, **kwargs).agg(
data_agg,
split_out=split_out,
)
return aggregated.set_crs(self.crs)
def sjoin(self, df, how="inner", predicate="intersects"):
"""
Spatial join of two GeoDataFrames.
Parameters
----------
df : geopandas or dask_geopandas GeoDataFrame
If a geopandas.GeoDataFrame is passed, it is considered as a
dask_geopandas.GeoDataFrame with 1 partition (without spatial
partitioning information).
how : string, default 'inner'
The type of join. Currently only 'inner' is supported.
predicate : string, default 'intersects'
Binary predicate how to match corresponding rows of the left and right
GeoDataFrame. Possible values: 'contains', 'contains_properly',
'covered_by', 'covers', 'crosses', 'intersects', 'overlaps',
'touches', 'within'.
Returns
-------
dask_geopandas.GeoDataFrame
Notes
-----
If both the left and right GeoDataFrame have spatial partitioning
information available (the ``spatial_partitions`` attribute is set),
the output partitions are determined based on intersection of the
spatial partitions. In all other cases, the output partitions are
all combinations (cartesian/cross product) of all input partition
of the left and right GeoDataFrame.
"""
return dask_geopandas.sjoin(self, df, how=how, predicate=predicate)
def spatial_shuffle(
self,
by="hilbert",
level=None,
calculate_partitions=True,
npartitions=None,
divisions=None,
**kwargs,
):
"""
Shuffle the data into spatially consistent partitions.
This realigns the dataset to be spatially sorted, i.e. geometries that are
spatially near each other will be within the same partition. This is
useful especially for overlay operations like a spatial join as it reduces the
number of interactions between individual partitions.
The spatial information is stored in the index and will replace the existing
index.
Note that ``spatial_shuffle`` uses ``set_index`` under the hood and comes with
all its potential performance drawbacks.
Parameters
----------
by : string (default 'hilbert')
Spatial sorting method, one of {'hilbert', 'morton', 'geohash'}. See
``hilbert_distance``, ``morton_distance`` and ``geohash`` methods for
details.
level : int (default None)
Level (precision) of the Hilbert and Morton
curves used as a sorting method. Defaults to 16. Does not have an effect for
the ``'geohash'`` option.
calculate_partitions : bool (default True)
Calculate new spatial partitions after shuffling
npartitions : int, None, or 'auto'
The ideal number of output partitions. If None, use the same as the input.
If 'auto' then decide by memory use. Only used when divisions is not given.
If divisions is given, the number of output partitions will be
len(divisions) - 1.
divisions: list, optional
The “dividing lines” used to split the new index into partitions. Needs to
match the values returned by the sorting method.
**kwargs
Keyword arguments passed to ``set_index``.
Returns
-------
dask_geopandas.GeoDataFrame
Notes
-----
This method, similarly to ``calculate_spatial_partitions``, is computed
partially eagerly as it needs to calculate the distances for all existing
partitions before it can determine the divisions for the new
spatially-shuffled partitions.
"""
if level is None:
level = 16
if by == "hilbert":
by = self.hilbert_distance(level=level)
elif by == "morton":
by = self.morton_distance(level=level)
elif by == "geohash":
by = self.geohash(as_string=False)
else:
raise ValueError(
f"'{by}' is not supported. Use one of ['hilbert', 'morton, 'geohash']."
)
sorted_ddf = self.set_index(
by,
sorted=False,
npartitions=npartitions,
divisions=divisions,
inplace=False,
**kwargs,
)
if calculate_partitions:
sorted_ddf.calculate_spatial_partitions()
return sorted_ddf
from_geopandas = dd.from_pandas
def from_dask_dataframe(df, geometry=None):
"""
Create GeoDataFrame from dask DataFrame.
Parameters
----------
df : dask DataFrame
geometry : str or array-like, optional
If a string, the column to use as geometry. By default, it will look
for a column named "geometry". If array-like or dask (Geo)Series,
the values will be set as 'geometry' column on the GeoDataFrame.
"""
return df.map_partitions(geopandas.GeoDataFrame, geometry=geometry)
@derived_from(geopandas)
def points_from_xy(df, x="x", y="y", z="z", crs=None):
"""Convert dask.dataframe of x and y (and optionally z) values to a GeoSeries."""
def func(data, x, y, z):
return geopandas.GeoSeries(
geopandas.points_from_xy(
data[x], data[y], data[z] if z in df.columns else None, crs=crs
),
index=data.index,
)
return df.map_partitions(
func, x, y, z, meta=geopandas.GeoSeries(), token="points_from_xy"
)
for name in [
"area",
"geom_type",
"type",
"length",
"is_valid",
"is_empty",
"is_simple",
"is_ring",
"has_z",
"interiors",
"bounds",
]:
_Frame._bind_property(name)
for name in [
"boundary",
"centroid",
"convex_hull",
"envelope",
"exterior",
]:
# TODO actually calculate envelope / convex_hull of the spatial partitions
# for some of those
_Frame._bind_property(name, preserve_spatial_partitions=True)
for name in [
"geometry",
"x",
"y",
"z",
]:
GeoSeries._bind_property(name)
for name in [
"contains",
"geom_equals",
"geom_almost_equals",
"crosses",
"disjoint",
"intersects",
"overlaps",
"touches",
"within",
"covers",
"covered_by",
]:
meth = getattr(geopandas.base.GeoPandasBase, name)
_Frame._bind_elemwise_comparison_method(
name, meth, original=geopandas.base.GeoPandasBase
)
for name in [
"distance",
"difference",
"symmetric_difference",
"union",
"intersection",
"relate",
"project",
]:
meth = getattr(geopandas.base.GeoPandasBase, name)
_Frame._bind_elemwise_operator_method(
name, meth, original=geopandas.base.GeoPandasBase
)
dd.core.DataFrame.set_geometry = GeoDataFrame.set_geometry
# Coodinate indexer (.cx)
def _cx_part(df, bbox):
idx = df.intersects(bbox)
return df[idx]
class _CoordinateIndexer(object):
def __init__(self, obj):
self.obj = obj
def __getitem__(self, key):
obj = self.obj
xs, ys = key
# handle numeric values as x and/or y coordinate index
if type(xs) is not slice:
xs = slice(xs, xs)
if type(ys) is not slice:
ys = slice(ys, ys)
if xs.step is not None or ys.step is not None:
raise ValueError("Slice step not supported.")
if self.obj.spatial_partitions is not None:
xmin, ymin, xmax, ymax = obj.spatial_partitions.total_bounds
bbox = box(
xs.start if xs.start is not None else xmin,
ys.start if ys.start is not None else ymin,
xs.stop if xs.stop is not None else xmax,
ys.stop if ys.stop is not None else ymax,
)
partition_idx = np.nonzero(
np.asarray(self.obj.spatial_partitions.intersects(bbox))
)[0]
else:
raise NotImplementedError(
"Not yet implemented if the GeoDataFrame has no known spatial "
"partitions (you can call the 'calculate_spatial_partitions' method "
"to set it)"
)
name = "cx-%s" % tokenize(key, self.obj)
if len(partition_idx):
# construct graph (based on LocIndexer from dask)
dsk = {}
for i, part in enumerate(partition_idx):
dsk[name, i] = (_cx_part, (self.obj._name, part), bbox)
divisions = [self.obj.divisions[i] for i in partition_idx] + [
self.obj.divisions[partition_idx[-1] + 1]
]
else:
# TODO can a dask dataframe have 0 partitions?
dsk = {(name, 0): self.obj._meta.head(0)}
divisions = [None, None]
graph = HighLevelGraph.from_collections(name, dsk, dependencies=[self.obj])
return new_dd_object(graph, name, meta=self.obj._meta, divisions=divisions)