async def test_assign_gpu_tasks(actor_pool): pool, session_id, assigner_ref, cluster_api, meta_api = actor_pool input1 = TensorFetch(key='a', source_key='a', dtype=np.dtype(int)).new_chunk([]) input2 = TensorFetch(key='b', source_key='b', dtype=np.dtype(int)).new_chunk([]) result_chunk = TensorTreeAdd(args=[input1, input2], gpu=True) \ .new_chunk([input1, input2]) chunk_graph = ChunkGraph([result_chunk]) chunk_graph.add_node(input1) chunk_graph.add_node(input2) chunk_graph.add_node(result_chunk) chunk_graph.add_edge(input1, result_chunk) chunk_graph.add_edge(input2, result_chunk) await meta_api.set_chunk_meta(input1, memory_size=200, store_size=200, bands=[('address0', 'numa-0')]) await meta_api.set_chunk_meta(input2, memory_size=200, store_size=200, bands=[('address0', 'numa-0')]) subtask = Subtask('test_task', session_id, chunk_graph=chunk_graph) [result] = await assigner_ref.assign_subtasks([subtask]) assert result[1].startswith('gpu')
async def test_execute_tensor(actor_pool): pool, session_id, meta_api, storage_api, execution_ref = actor_pool data1 = np.random.rand(10, 10) data2 = np.random.rand(10, 10) input1 = TensorFetch(key='input1', source_key='input2', dtype=np.dtype(int)).new_chunk([]) input2 = TensorFetch(key='input2', source_key='input2', dtype=np.dtype(int)).new_chunk([]) result_chunk = TensorTreeAdd(args=[input1, input2]) \ .new_chunk([input1, input2], shape=data1.shape, dtype=data1.dtype) await meta_api.set_chunk_meta(input1, memory_size=data1.nbytes, store_size=data1.nbytes, bands=[(pool.external_address, 'numa-0')]) await meta_api.set_chunk_meta(input2, memory_size=data1.nbytes, store_size=data2.nbytes, bands=[(pool.external_address, 'numa-0')]) # todo use different storage level when storage ready await storage_api.put(input1.key, data1) await storage_api.put(input2.key, data2) chunk_graph = ChunkGraph([result_chunk]) chunk_graph.add_node(input1) chunk_graph.add_node(input2) chunk_graph.add_node(result_chunk) chunk_graph.add_edge(input1, result_chunk) chunk_graph.add_edge(input2, result_chunk) subtask = Subtask('test_task', session_id=session_id, chunk_graph=chunk_graph) await execution_ref.run_subtask(subtask, 'numa-0', pool.external_address) # check if results are correct result = await storage_api.get(result_chunk.key) np.testing.assert_array_equal(data1 + data2, result) # check if quota computations are correct quota_ref = await mo.actor_ref(QuotaActor.gen_uid('numa-0'), address=pool.external_address) [quota] = await quota_ref.get_batch_quota_reqs() assert quota[(subtask.subtask_id, subtask.subtask_id)] == data1.nbytes # check if metas are correct result_meta = await meta_api.get_chunk_meta(result_chunk.key) assert result_meta['object_id'] == result_chunk.key assert result_meta['shape'] == result.shape
def _build_test_graph(data_list): from mars.tensor.fetch import TensorFetch from mars.tensor.arithmetic import TensorTreeAdd inputs = [] for idx, d in enumerate(data_list): chunk_key = f'chunk-{random.randint(0, 999)}-{idx}' fetch_chunk = TensorFetch(to_fetch_key=chunk_key, dtype=d.dtype) \ .new_chunk([], shape=d.shape, _key=chunk_key) inputs.append(fetch_chunk) add_chunk = TensorTreeAdd(args=inputs, dtype=data_list[0].dtype) \ .new_chunk(inputs, shape=data_list[0].shape) exec_graph = ChunkGraph([add_chunk.data]) exec_graph.add_node(add_chunk.data) for input_chunk in inputs: exec_graph.add_node(input_chunk.data) exec_graph.add_edge(input_chunk.data, add_chunk.data) return exec_graph, inputs, add_chunk
async def test_cancel_without_kill(actor_pool): pool, session_id, meta_api, storage_api, execution_ref = actor_pool def delay_fun(delay): import mars time.sleep(delay) mars._slot_marker = 1 return delay def check_fun(): import mars return getattr(mars, '_slot_marker', False) remote_result = RemoteFunction(function=delay_fun, function_args=[2], function_kwargs={}).new_chunk([]) chunk_graph = ChunkGraph([remote_result]) chunk_graph.add_node(remote_result) subtask = Subtask(f'test_task_{uuid.uuid4()}', session_id=session_id, chunk_graph=chunk_graph) aiotask = asyncio.create_task(execution_ref.run_subtask( subtask, 'numa-0', pool.external_address)) await asyncio.sleep(0.5) await execution_ref.cancel_subtask(subtask.subtask_id, kill_timeout=1) with pytest.raises(asyncio.CancelledError): await asyncio.wait_for(aiotask, timeout=30) remote_result = RemoteFunction(function=check_fun, function_args=[], function_kwargs={}).new_chunk([]) chunk_graph = ChunkGraph([remote_result]) chunk_graph.add_node(remote_result) subtask = Subtask(f'test_task_{uuid.uuid4()}', session_id=session_id, chunk_graph=chunk_graph) await execution_ref.run_subtask( subtask, 'numa-0', pool.external_address) # check if results are correct assert await storage_api.get(remote_result.key)
async def test_assigner(actor_pool): pool, session_id, assigner_ref, meta_api = actor_pool input1 = TensorFetch(key='a', source_key='a', dtype=np.dtype(int)).new_chunk([]) input2 = TensorFetch(key='b', source_key='b', dtype=np.dtype(int)).new_chunk([]) input3 = TensorFetch(key='c', source_key='c', dtype=np.dtype(int)).new_chunk([]) result_chunk = TensorTreeAdd(args=[input1, input2, input3]) \ .new_chunk([input1, input2, input3]) chunk_graph = ChunkGraph([result_chunk]) chunk_graph.add_node(input1) chunk_graph.add_node(input2) chunk_graph.add_node(input3) chunk_graph.add_node(result_chunk) chunk_graph.add_edge(input1, result_chunk) chunk_graph.add_edge(input2, result_chunk) chunk_graph.add_edge(input3, result_chunk) await meta_api.set_chunk_meta(input1, memory_size=200, store_size=200, bands=[('address0', 'numa-0')]) await meta_api.set_chunk_meta(input2, memory_size=400, store_size=400, bands=[('address1', 'numa-0')]) await meta_api.set_chunk_meta(input3, memory_size=400, store_size=400, bands=[('address2', 'numa-0')]) subtask = Subtask('test_task', session_id, chunk_graph=chunk_graph) [result] = await assigner_ref.assign_subtasks([subtask]) assert result in (('address1', 'numa-0'), ('address2', 'numa-0'))
async def test_execute_with_cancel(actor_pool, cancel_phase): pool, session_id, meta_api, storage_api, execution_ref = actor_pool # config for different phases ref_to_delay = None if cancel_phase == 'prepare': ref_to_delay = await mo.actor_ref(StorageManagerActor.default_uid(), address=pool.external_address) elif cancel_phase == 'quota': ref_to_delay = await mo.actor_ref(QuotaActor.gen_uid('numa-0'), address=pool.external_address) elif cancel_phase == 'slot': ref_to_delay = await mo.actor_ref( BandSlotManagerActor.gen_uid('numa-0'), address=pool.external_address) if ref_to_delay: await ref_to_delay.set_delay_fetch_time(100) def delay_fun(delay, _inp1): time.sleep(delay) return delay input1 = TensorFetch(key='input1', source_key='input1', dtype=np.dtype(int)).new_chunk([]) remote_result = RemoteFunction(function=delay_fun, function_args=[100, input1], function_kwargs={}, n_output=1) \ .new_chunk([input1]) data1 = np.random.rand(10, 10) await meta_api.set_chunk_meta(input1, memory_size=data1.nbytes, store_size=data1.nbytes, bands=[(pool.external_address, 'numa-0')]) await storage_api.put(input1.key, data1) chunk_graph = ChunkGraph([remote_result]) chunk_graph.add_node(input1) chunk_graph.add_node(remote_result) chunk_graph.add_edge(input1, remote_result) subtask = Subtask(f'test_task_{uuid.uuid4()}', session_id=session_id, chunk_graph=chunk_graph) aiotask = asyncio.create_task( execution_ref.run_subtask(subtask, 'numa-0', pool.external_address)) await asyncio.sleep(1) with Timer() as timer: await execution_ref.cancel_subtask(subtask.subtask_id, kill_timeout=1) with pytest.raises(asyncio.CancelledError): await asyncio.wait_for(aiotask, timeout=30) assert timer.duration < 6 # check for different phases if ref_to_delay is not None: assert await ref_to_delay.get_is_cancelled() await ref_to_delay.set_delay_fetch_time(0) # test if slot is restored remote_tileable = mr.spawn(delay_fun, args=(0.5, None)) graph = TileableGraph([remote_tileable.data]) next(TileableGraphBuilder(graph).build()) chunk_graph = next(ChunkGraphBuilder(graph, fuse_enabled=False).build()) subtask = Subtask(f'test_task2_{uuid.uuid4()}', session_id=session_id, chunk_graph=chunk_graph) await asyncio.wait_for(execution_ref.run_subtask(subtask, 'numa-0', pool.external_address), timeout=30)
def test_fuse(): """ test compose in build graph and optimize """ r""" graph(@: node, #: composed_node): @ --> @ --> @ ========> # """ chunks = [ TensorTreeAdd(args=[], _key=str(n)).new_chunk(None, None).data for n in range(3) ] graph = ChunkGraph([]) for c in chunks: graph.add_node(c) graph.add_edge(chunks[0], chunks[1]) graph.add_edge(chunks[1], chunks[2]) graph2 = graph.copy() graph2._result_chunks = [chunks[2]] _, fused_nodes = Fusion(graph2).fuse() assert fused_nodes[0].composed == chunks[:3] # make the middle one as result chunk, thus the graph cannot be composed graph3 = graph.copy() graph3._result_chunks = [chunks[1]] _, fused_nodes = Fusion(graph3).fuse() assert fused_nodes[0].composed == chunks[:2] r""" graph(@: node, #: composed_node): @ @ @ @ \ / \ / @ --> @ ========> # / \ / \ @ @ @ @ """ chunks = [ TensorTreeAdd(args=[], _key=str(n)).new_chunk(None, None).data for n in range(6) ] graph = ChunkGraph([chunks[4], chunks[5]]) for c in chunks: graph.add_node(c) chunks[2].op._inputs = [chunks[0], chunks[1]] chunks[3].op._inputs = [chunks[2]] chunks[4].op._inputs = [chunks[3]] chunks[5].op._inputs = [chunks[3]] graph.add_edge(chunks[0], chunks[2]) graph.add_edge(chunks[1], chunks[2]) graph.add_edge(chunks[2], chunks[3]) graph.add_edge(chunks[3], chunks[4]) graph.add_edge(chunks[3], chunks[5]) _, fused_nodes = Fusion(graph).fuse() assert fused_nodes[0].composed == chunks[2:4] # to make sure the predecessors and successors of compose are right # 0 and 1's successors must be composed assert fused_nodes[0] in graph.successors(chunks[0]) assert fused_nodes[0] in graph.successors(chunks[1]) # check composed's inputs assert chunks[0] in fused_nodes[0].inputs assert chunks[1] in fused_nodes[0].inputs # check composed's predecessors assert chunks[0] in graph.predecessors(fused_nodes[0]) assert chunks[1] in graph.predecessors(fused_nodes[0]) # check 4 and 5's inputs assert fused_nodes[0] in graph.successors(fused_nodes[0])[0].inputs assert fused_nodes[0] in graph.successors(fused_nodes[0])[0].inputs # check 4 and 5's predecessors assert fused_nodes[0] in graph.predecessors(chunks[4]) assert fused_nodes[0] in graph.predecessors(chunks[5])
async def test_assign_cpu_tasks(actor_pool): pool, session_id, assigner_ref, cluster_api, meta_api = actor_pool input1 = TensorFetch(key='a', source_key='a', dtype=np.dtype(int)).new_chunk([]) input2 = TensorFetch(key='b', source_key='b', dtype=np.dtype(int)).new_chunk([]) input3 = TensorFetch(key='c', source_key='c', dtype=np.dtype(int)).new_chunk([]) result_chunk = TensorTreeAdd(args=[input1, input2, input3]) \ .new_chunk([input1, input2, input3]) chunk_graph = ChunkGraph([result_chunk]) chunk_graph.add_node(input1) chunk_graph.add_node(input2) chunk_graph.add_node(input3) chunk_graph.add_node(result_chunk) chunk_graph.add_edge(input1, result_chunk) chunk_graph.add_edge(input2, result_chunk) chunk_graph.add_edge(input3, result_chunk) await meta_api.set_chunk_meta(input1, memory_size=200, store_size=200, bands=[('address0', 'numa-0')]) await meta_api.set_chunk_meta(input2, memory_size=400, store_size=400, bands=[('address1', 'numa-0')]) await meta_api.set_chunk_meta(input3, memory_size=400, store_size=400, bands=[('address2', 'numa-0')]) await cluster_api.set_node_status(node='address1', role=NodeRole.WORKER, status=NodeStatus.STOPPING) await cluster_api.set_node_status(node='address3', role=NodeRole.WORKER, status=NodeStatus.STOPPING) subtask = Subtask('test_task', session_id, chunk_graph=chunk_graph) [result] = await assigner_ref.assign_subtasks([subtask]) assert result in (('address0', 'numa-0'), ('address2', 'numa-0')) subtask.expect_bands = [('address0', 'numa-0')] [result] = await assigner_ref.assign_subtasks([subtask]) assert result == ('address0', 'numa-0') subtask.expect_bands = [('address0', 'numa-0'), ('address1', 'numa-0')] [result] = await assigner_ref.assign_subtasks([subtask]) assert result == ('address0', 'numa-0') subtask.expect_bands = [('address1', 'numa-0')] [result] = await assigner_ref.assign_subtasks([subtask]) assert result in (('address0', 'numa-0'), ('address2', 'numa-0')) result_chunk.op.gpu = True subtask = Subtask('test_task', session_id, chunk_graph=chunk_graph) with pytest.raises(NoMatchingSlots) as err: await assigner_ref.assign_subtasks([subtask]) assert 'gpu' in str(err.value)