-
Notifications
You must be signed in to change notification settings - Fork 122
/
Copy pathdask.py
152 lines (126 loc) · 5.26 KB
/
dask.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
#! /usr/bin/env python
# -*- coding: utf-8 -*-
#
# Copyright 2020-2023 Alibaba Group Holding Limited.
#
# 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.
#
import contextlib
import json
import numpy as np
import pandas as pd
import dask
import dask.array as da
import dask.dataframe as dd
from dask.distributed import Client # pylint: disable=no-name-in-module
import vineyard
from vineyard.core import context
from vineyard.data.dataframe import make_global_dataframe
from vineyard.data.tensor import make_global_tensor
def dask_array_builder(client, value, builder, **kw): # pylint: disable=unused-argument
def put_partition(v, block_id=None):
client = vineyard.connect()
obj_id = client.put(v, partition_index=block_id)
client.persist(obj_id)
return np.array([[int(obj_id)]])
_ = Client(kw['dask_scheduler']) # enforce distributed scheduling
blocks = value.map_blocks(put_partition, dtype=int).compute().flatten()
return make_global_tensor(client, blocks)
def dask_dataframe_builder(
client, value, builder, **kw
): # pylint: disable=unused-argument
def put_partition(v, partition_info=None):
client = vineyard.connect()
obj_id = client.put(v, partition_index=(partition_info['number'], 0))
client.persist(obj_id)
return pd.DataFrame([{'no': partition_info['number'], 'id': int(obj_id)}])
_ = Client(kw['dask_scheduler']) # enforce distributed scheduling
res = value.map_partitions(put_partition, meta={'no': int, 'id': int}).compute()
res = res.set_index('no')
blocks = [res.loc[i] for i in range(len(res))]
return make_global_dataframe(client, blocks)
def dask_array_resolver(obj, resolver, **kw): # pylint: disable=unused-argument
def get_partition(obj_id):
client = vineyard.connect()
np_value = client.get(obj_id)
return da.from_array(np_value)
meta = obj.meta
num = int(meta['partitions_-size'])
dask_client = Client(kw['dask_scheduler'])
futures = []
indices = []
with_index = True
for i in range(num):
ts = meta.get_member('partitions_-%d' % i)
instance_id = int(ts.meta['instance_id'])
partition_index = json.loads(ts.meta['partition_index_'])
if partition_index:
indices.append((partition_index[0], partition_index[1], i))
else:
with_index = False
futures.append(
# we require the 1-on-1 alignment of vineyard instances and dask workers.
# vineyard_sockets maps vineyard instance_ids into ipc_sockets, while
# dask_workers maps vineyard instance_ids into names of dask workers.
dask_client.submit(
get_partition, ts.meta.id, workers={kw['dask_workers'][instance_id]}
)
)
arrays = dask_client.gather(futures)
if with_index:
indices = list(sorted(indices))
nx = indices[-1][0] + 1
ny = indices[-1][1] + 1
assert nx * ny == num
rows = []
for i in range(nx):
cols = []
for j in range(ny):
cols.append(arrays[indices[i * ny + j][2]])
rows.append(da.hstack(cols))
return da.vstack(rows)
return da.vstack(arrays)
def dask_dataframe_resolver(obj, resolver, **kw): # pylint: disable=unused-argument
def get_partition(obj_id):
client = vineyard.connect()
df = client.get(obj_id)
return dd.from_pandas(df, npartitions=1)
meta = obj.meta
num = int(meta['partitions_-size'])
dask_client = Client(kw['dask_scheduler'])
futures = []
for i in range(num):
df = meta.get_member('partitions_-%d' % i)
instance_id = int(df.meta['instance_id'])
futures.append(
# we require the 1-on-1 alignment of vineyard instances and dask workers.
# vineyard_sockets maps vineyard instance_ids into ipc_sockets, while
# dask_workers maps vineyard instance_ids into names of dask workers.
dask_client.submit(
get_partition, df.meta.id, workers={kw['dask_workers'][instance_id]}
)
)
dfs = dask_client.gather(futures)
return dd.concat(dfs, axis=0)
def register_dask_types(builder_ctx=None, resolver_ctx=None):
if builder_ctx is not None:
builder_ctx.register(dask.array.Array, dask_array_builder)
builder_ctx.register(dask.dataframe.DataFrame, dask_dataframe_builder)
if resolver_ctx is not None:
resolver_ctx.register('vineyard::GlobalTensor', dask_array_resolver)
resolver_ctx.register('vineyard::GlobalDataFrame', dask_dataframe_resolver)
@contextlib.contextmanager
def dask_context():
with context() as (builder_ctx, resolver_ctx):
register_dask_types(builder_ctx, resolver_ctx)
yield builder_ctx, resolver_ctx