@@ -193,6 +193,17 @@ def set_chunk_broadcasts(self, session_id, chunk_key, broadcast_dests):
193
193
self ._meta_broadcasts [(session_id , chunk_key )] = \
194
194
[d for d in broadcast_dests if d != self .address ]
195
195
196
+ def batch_set_chunk_broadcasts (self , session_id , chunk_keys , broadcast_dests ):
197
+ """
198
+ Configure broadcast destinations in batch
199
+ :param session_id: session id
200
+ :param chunk_keys: chunk key
201
+ :param broadcast_dests:
202
+ :return:
203
+ """
204
+ for key , dests in zip (chunk_keys , broadcast_dests ):
205
+ self .set_chunk_broadcasts (session_id , key , dests )
206
+
196
207
def get_chunk_broadcasts (self , session_id , chunk_key ):
197
208
"""
198
209
Get chunk broadcast addresses, for test only
@@ -201,14 +212,16 @@ def get_chunk_broadcasts(self, session_id, chunk_key):
201
212
"""
202
213
return self ._meta_broadcasts .get ((session_id , chunk_key ))
203
214
204
- def set_chunk_meta (self , session_id , chunk_key , size = None , shape = None , workers = None ):
215
+ def set_chunk_meta (self , session_id , chunk_key , size = None , shape = None , workers = None ,
216
+ broadcast = True ):
205
217
"""
206
218
Update chunk meta in current storage
207
219
:param session_id: session id
208
220
:param chunk_key: chunk key
209
221
:param size: size of the chunk
210
222
:param shape: shape of the chunk
211
223
:param workers: workers holding the chunk
224
+ :param broadcast: broadcast meta into registered destinations
212
225
"""
213
226
query_key = (session_id , chunk_key )
214
227
# update input with existing value
@@ -238,24 +251,54 @@ def set_chunk_meta(self, session_id, chunk_key, size=None, shape=None, workers=N
238
251
239
252
# broadcast into pre-determined destinations
240
253
futures = []
241
- if query_key in self ._meta_broadcasts :
254
+ if broadcast and query_key in self ._meta_broadcasts :
242
255
for dest in self ._meta_broadcasts [query_key ]:
243
256
futures .append (
244
257
self .ctx .actor_ref (self .default_name (), address = dest )
245
- .cache_chunk_meta (session_id , chunk_key , meta , _wait = False , _tell = True )
258
+ .batch_cache_chunk_meta (session_id , [ chunk_key ], [ meta ] , _wait = False , _tell = True )
246
259
)
247
260
[f .result () for f in futures ]
248
261
249
- def cache_chunk_meta (self , session_id , chunk_key , meta ):
262
+ def batch_set_chunk_meta (self , session_id , keys , metas ):
263
+ """
264
+ Set chunk metas in batch
265
+ :param session_id: session id
266
+ :param keys: keys to set
267
+ :param metas: metas to set
268
+ """
269
+ query_dict = defaultdict (lambda : (list (), list ()))
270
+
271
+ for key , meta in zip (keys , metas ):
272
+ self .set_chunk_meta (session_id , key , size = meta .chunk_size , shape = meta .chunk_shape ,
273
+ workers = meta .workers , broadcast = False )
274
+ try :
275
+ dests = self ._meta_broadcasts [(session_id , key )]
276
+ except KeyError :
277
+ continue
278
+
279
+ for dest in dests :
280
+ query_dict [dest ][0 ].append (key )
281
+ query_dict [dest ][1 ].append (meta )
282
+
283
+ futures = []
284
+ for dest , (chunk_keys , metas ) in query_dict .items ():
285
+ futures .append (
286
+ self .ctx .actor_ref (self .default_name (), address = dest )
287
+ .batch_cache_chunk_meta (session_id , chunk_keys , metas , _wait = False , _tell = True )
288
+ )
289
+ [f .result () for f in futures ]
290
+
291
+ def batch_cache_chunk_meta (self , session_id , chunk_keys , metas ):
250
292
"""
251
293
Receive updates for caching
252
294
253
295
:param session_id: session id
254
- :param chunk_key : chunk key
255
- :param meta : meta data
296
+ :param chunk_keys : chunk keys
297
+ :param metas : meta data
256
298
"""
257
- query_key = (session_id , chunk_key )
258
- self ._meta_cache [query_key ] = meta
299
+ for chunk_key , meta in zip (chunk_keys , metas ):
300
+ query_key = (session_id , chunk_key )
301
+ self ._meta_cache [query_key ] = meta
259
302
260
303
def get_chunk_meta (self , session_id , chunk_key ):
261
304
"""
@@ -361,6 +404,17 @@ def set_chunk_broadcasts(self, session_id, chunk_key, broadcast_dests):
361
404
self .ctx .actor_ref (LocalChunkMetaActor .default_name (), address = addr ) \
362
405
.set_chunk_broadcasts (session_id , chunk_key , broadcast_dests , _tell = True , _wait = False )
363
406
407
+ def batch_set_chunk_broadcasts (self , session_id , chunk_keys , broadcast_dests ):
408
+ query_chunk = defaultdict (lambda : (list (), list ()))
409
+ for key , dests in zip (chunk_keys , broadcast_dests ):
410
+ addr = self .get_scheduler ((session_id , key ))
411
+ query_chunk [addr ][0 ].append (key )
412
+ query_chunk [addr ][1 ].append (dests )
413
+
414
+ for addr , (chunk_keys , dest_groups ) in query_chunk .items ():
415
+ self .ctx .actor_ref (LocalChunkMetaActor .default_name (), address = addr ) \
416
+ .batch_set_chunk_broadcasts (session_id , chunk_keys , dest_groups )
417
+
364
418
def set_chunk_meta (self , session_id , chunk_key , size = None , shape = None , workers = None ):
365
419
"""
366
420
Update chunk metadata
0 commit comments