-
-
Notifications
You must be signed in to change notification settings - Fork 115
Description
I'm trying to follow the Minimal Dask Azure Tutorial. I used an existing resource group with a new vnet and security group created as described in the tutorial (except that I forgot to add the subnet flag).
However, the following code fails:
from dask_cloudprovider.azure import AzureVMCluster
cluster = AzureVMCluster(resource_group="VMs_kx",
location="northeurope",
vnet="dasknet2",
security_group="dasksecuritygroup2",
n_workers=1)
What happened:
I receive the following stacktrace. There is two things that should be improved:
- The
ValueError: not enough values to unpack (expected at least 1, got 0)
error seems to be caused by the CLI not finding a subnet. - During the exception, another exception occures due to the missing
nic
property.
Creating scheduler instance
2022-07-11 09:31:09,925 - distributed.deploy.spec - WARNING - Cluster closed without starting up
Terminated VM dask-0806ddcf-scheduler
Removed disks for VM dask-0806ddcf-scheduler
---------------------------------------------------------------------------
ValueError Traceback (most recent call last)
File /opt/conda/lib/python3.9/site-packages/distributed/deploy/spec.py:298, in SpecCluster._start(self)
297 self.scheduler = cls(**self.scheduler_spec.get("options", {}))
--> 298 self.scheduler = await self.scheduler
299 self.scheduler_comm = rpc(
300 getattr(self.scheduler, "external_address", None)
301 or self.scheduler.address,
302 connection_args=self.security.get_connection_args("client"),
303 )
File /opt/conda/lib/python3.9/site-packages/distributed/deploy/spec.py:63, in ProcessInterface.__await__.<locals>._()
62 if self.status == Status.created:
---> 63 await self.start()
64 assert self.status == Status.running
File /opt/conda/lib/python3.9/site-packages/dask_cloudprovider/generic/vmcluster.py:88, in SchedulerMixin.start(self)
87 self.cluster._log("Creating scheduler instance")
---> 88 ip = await self.create_vm()
89 self.address = f"{self.cluster.protocol}://{ip}:{self.port}"
File /opt/conda/lib/python3.9/site-packages/dask_cloudprovider/azure/azurevm.py:75, in AzureVM.create_vm(self)
74 async def create_vm(self):
---> 75 [subnet_info, *_] = await self.cluster.call_async(
76 self.cluster.network_client.subnets.list,
77 self.cluster.resource_group,
78 self.cluster.vnet,
79 )
81 nic_parameters = {
82 "location": self.location,
83 "ip_configurations": [
(...)
99 "tags": self.cluster.get_tags(),
100 }
ValueError: not enough values to unpack (expected at least 1, got 0)
During handling of the above exception, another exception occurred:
AttributeError Traceback (most recent call last)
Input In [2], in <cell line: 1>()
----> 1 cluster = AzureVMCluster(resource_group="VMs_kx",
2 location="northeurope",
3 vnet="dasknet2",
4 security_group="dasksecuritygroup2",
5 n_workers=1)
File /opt/conda/lib/python3.9/site-packages/dask_cloudprovider/azure/azurevm.py:559, in AzureVMCluster.__init__(self, location, resource_group, vnet, security_group, public_ingress, vm_size, scheduler_vm_size, vm_image, disk_size, bootstrap, auto_shutdown, docker_image, debug, marketplace_plan, **kwargs)
553 self.scheduler_options = {
554 "vm_size": self.scheduler_vm_size,
555 "public_ingress": self.public_ingress,
556 **self.options,
557 }
558 self.worker_options = {"vm_size": self.vm_size, **self.options}
--> 559 super().__init__(debug=debug, **kwargs)
File /opt/conda/lib/python3.9/site-packages/dask_cloudprovider/generic/vmcluster.py:297, in VMCluster.__init__(self, n_workers, worker_class, worker_options, scheduler_options, docker_image, docker_args, extra_bootstrap, env_vars, security, protocol, debug, **kwargs)
294 self.worker_options["extra_bootstrap"] = extra_bootstrap
295 self.uuid = str(uuid.uuid4())[:8]
--> 297 super().__init__(**kwargs, security=self.security)
File /opt/conda/lib/python3.9/site-packages/distributed/deploy/spec.py:264, in SpecCluster.__init__(self, workers, scheduler, worker, asynchronous, loop, security, silence_logs, name, shutdown_on_close, scheduler_sync_interval)
262 if not self.asynchronous:
263 self._loop_runner.start()
--> 264 self.sync(self._start)
265 try:
266 self.sync(self._correct_state)
File /opt/conda/lib/python3.9/site-packages/distributed/utils.py:336, in SyncMethodMixin.sync(self, func, asynchronous, callback_timeout, *args, **kwargs)
334 return future
335 else:
--> 336 return sync(
337 self.loop, func, *args, callback_timeout=callback_timeout, **kwargs
338 )
File /opt/conda/lib/python3.9/site-packages/distributed/utils.py:403, in sync(loop, func, callback_timeout, *args, **kwargs)
401 if error:
402 typ, exc, tb = error
--> 403 raise exc.with_traceback(tb)
404 else:
405 return result
File /opt/conda/lib/python3.9/site-packages/distributed/utils.py:376, in sync.<locals>.f()
374 future = asyncio.wait_for(future, callback_timeout)
375 future = asyncio.ensure_future(future)
--> 376 result = yield future
377 except Exception:
378 error = sys.exc_info()
File /opt/conda/lib/python3.9/site-packages/tornado/gen.py:762, in Runner.run(self)
759 exc_info = None
761 try:
--> 762 value = future.result()
763 except Exception:
764 exc_info = sys.exc_info()
File /opt/conda/lib/python3.9/site-packages/dask_cloudprovider/generic/vmcluster.py:339, in VMCluster._start(self)
329 self.worker_spec = {
330 self._new_worker_name(i): self.new_spec for i in range(self._n_workers)
331 }
333 with warn_on_duration(
334 "10s",
335 "Creating your cluster is taking a surprisingly long time. "
336 "This is likely due to pending resources. "
337 "Hang tight! ",
338 ):
--> 339 await super()._start()
File /opt/conda/lib/python3.9/site-packages/distributed/deploy/spec.py:307, in SpecCluster._start(self)
305 except Exception as e: # pragma: no cover
306 self.status = Status.failed
--> 307 await self._close()
308 raise RuntimeError(f"Cluster failed to start: {e}") from e
File /opt/conda/lib/python3.9/site-packages/distributed/deploy/spec.py:423, in SpecCluster._close(self)
420 else:
421 logger.warning("Cluster closed without starting up")
--> 423 await self.scheduler.close()
424 for w in self._created:
425 assert w.status in {
426 Status.closing,
427 Status.closed,
428 Status.failed,
429 }, w.status
File /opt/conda/lib/python3.9/site-packages/dask_cloudprovider/generic/vmcluster.py:60, in VMInterface.close(self)
58 async def close(self):
59 """Destroy a VM."""
---> 60 await self.destroy_vm()
61 await super().close()
File /opt/conda/lib/python3.9/site-packages/dask_cloudprovider/azure/azurevm.py:227, in AzureVM.destroy_vm(self)
218 await self.cluster.call_async(
219 self.cluster.compute_client.disks.begin_delete(
220 self.cluster.resource_group,
221 disk.name,
222 ).wait
223 )
224 self.cluster._log(f"Removed disks for VM {self.name}")
225 await self.cluster.call_async(
226 self.cluster.network_client.network_interfaces.begin_delete(
--> 227 self.cluster.resource_group, self.nic.name
228 ).wait
229 )
230 self.cluster._log("Deleted network interface")
231 if self.public_ingress:
AttributeError: 'AzureVMScheduler' object has no attribute 'nic'
Manually checking for subnets also returns an empty list
(base) root@mccprodkx:/home/mccdev/windows# az network vnet subnet list --resource-group VMs_kx --vnet-name dasknet2
[]
What you expected to happen:
The code should print a error message indicating that there are no subnets to be found in the the resource group and network security group. I would suggest to manually check the subnets using
az network vnet subnet list --resource-group <resource group name>--vnet-name <network name>
And in case of a missing subnet suggest adding a subnet, or automatically add one as described in the tutorial.
Also, destroy_vm
should handle the case of a missing nic. Or should not be called in case the VM was not successfully created.
Minimal Complete Verifiable Example:
As described above.
Anything else we need to know?:
Environment:
- Dask version: dask==2022.6.1, dask-cloudprovider==2022.5.0
- Python version: 3.9
- Operating System: Docker VM on Windows
- Install method (conda, pip, source): pip