-
Notifications
You must be signed in to change notification settings - Fork 9
/
Copy pathcore.py
208 lines (176 loc) · 5.42 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
from __future__ import annotations
try:
from os import register_at_fork
except ImportError:
register_at_fork = None
import atexit
import weakref
from collections.abc import Mapping
from copy import copy
from functools import lru_cache
from math import ceil
from typing import Any
import pymongo
from bson import ObjectId
from dask.bag import Bag
from dask.base import tokenize
from dask.graph_manipulation import checkpoint
from ._version import __version__
appname = f"dask-mongo/{__version__}"
_CACHE_SIZE = 16
def _recursive_tupling(item):
if isinstance(item, list):
return tuple([_recursive_tupling(i) for i in item])
if isinstance(item, Mapping):
return tuple(
[(_recursive_tupling(k), _recursive_tupling(v)) for k, v in item.items()]
)
else:
return item
class _FrozenKwargs(dict):
def __hash__(self):
return hash(
frozenset(
[
(_recursive_tupling(k), _recursive_tupling(v))
for k, v in self.items()
]
)
)
@lru_cache(_CACHE_SIZE, typed=True)
def _cache_inner(kwargs):
client = pymongo.MongoClient(appname=appname, **kwargs)
atexit.register(weakref.WeakMethod(client.close))
return client
def _clear_cache():
_cache_inner.cache_clear()
if register_at_fork:
register_at_fork(after_in_child=_clear_cache)
def _get_client(kwargs):
return _cache_inner(_FrozenKwargs(kwargs))
def write_mongo(
values: list[dict],
connection_kwargs: dict[str, Any],
database: str,
collection: str,
) -> None:
mongo_client = _get_client(connection_kwargs)
coll = mongo_client[database][collection]
# `insert_many` will mutate its input by inserting a "_id" entry.
# This can lead to confusing results; pass copies to it to preserve the input.
values = [copy(v) for v in values]
coll.insert_many(values)
def to_mongo(
bag: Bag,
database: str,
collection: str,
*,
connection_kwargs: dict[str, Any] = None,
compute: bool = True,
compute_kwargs: dict[str, Any] = None,
) -> Any:
"""Write a Dask Bag to a Mongo database.
Parameters
----------
bag:
Dask Bag to write into the database.
database : str
Name of the database to write to. If it does not exists it will be created.
collection : str
Name of the collection within the database to write to.
If it does not exists it will be created.
connection_kwargs : dict
Arguments to pass to ``MongoClient``.
compute : bool, optional
If true, immediately executes. If False, returns a delayed
object, which can be computed at a later time.
compute_kwargs : dict, optional
Options to be passed in to the compute method
Returns
-------
If compute=True, block until computation is done, then return None.
If compute=False, immediately return a dask.delayed object.
"""
partials = bag.map_partitions(
write_mongo, connection_kwargs or {}, database, collection
)
collect = checkpoint(partials)
if compute:
return collect.compute(**compute_kwargs or {})
else:
return collect
def fetch_mongo(
connection_kwargs: dict[str, Any],
database: str,
collection: str,
match: dict[str, Any],
id_min: ObjectId,
id_max: ObjectId,
include_last: bool,
) -> list[dict[str, Any]]:
match2 = {"_id": {"$gte": id_min, "$lte" if include_last else "$lt": id_max}}
mongo_client = _get_client(connection_kwargs)
coll = mongo_client[database][collection]
return list(coll.aggregate([{"$match": match}, {"$match": match2}]))
def read_mongo(
database: str,
collection: str,
chunksize: int,
*,
connection_kwargs: dict[str, Any] = None,
match: dict[str, Any] = None,
):
"""Read data from a Mongo database into a Dask Bag.
Parameters
----------
database:
Name of the database to read from
collection:
Name of the collection within the database to read from
chunksize:
Number of elements desired per partition.
connection_kwargs:
Connection arguments to pass to ``MongoClient``
match:
MongoDB match query, used to filter the documents in the collection. If omitted,
this function will load all the documents in the collection.
"""
if not connection_kwargs:
connection_kwargs = {}
if not match:
match = {}
mongo_client = _get_client(connection_kwargs)
coll = mongo_client[database][collection]
nrows = next(
(
coll.aggregate(
[
{"$match": match},
{"$count": "count"},
]
)
)
)["count"]
npartitions = int(ceil(nrows / chunksize))
partitions_ids = list(
coll.aggregate(
[
{"$match": match},
{"$bucketAuto": {"groupBy": "$_id", "buckets": npartitions}},
],
allowDiskUse=True,
)
)
common_args = (connection_kwargs, database, collection, match)
name = "read_mongo-" + tokenize(common_args, chunksize)
dsk = {
(name, i): (
fetch_mongo,
*common_args,
partition["_id"]["min"],
partition["_id"]["max"],
i == len(partitions_ids) - 1,
)
for i, partition in enumerate(partitions_ids)
}
return Bag(dsk, name, len(partitions_ids))