Skip to content
Merged
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
42 changes: 38 additions & 4 deletions python_modules/libraries/dagster-dask/dagster_dask/executor.py
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,21 @@
'pbs': Field(
Permissive(), is_required=False, description='PBS cluster configuration.'
),
'moab': Field(
Permissive(), is_required=False, description='Moab cluster configuration.'
),
'sge': Field(
Permissive(), is_required=False, description='SGE cluster configuration.'
),
'lsf': Field(
Permissive(), is_required=False, description='LSF cluster configuration.'
),
'slurm': Field(
Permissive(), is_required=False, description='SLURM cluster configuration.'
),
'oar': Field(
Permissive(), is_required=False, description='OAR cluster configuration.'
),
'kube': Field(
Permissive(),
is_required=False,
Expand All @@ -55,7 +70,7 @@ def dask_executor(init_context):

cluster:
{
local?: # The cluster type, one of the following ('local', 'yarn', 'ssh', 'pbs', 'kube').
local?: # The cluster type, one of the following ('local', 'yarn', 'ssh', 'pbs', 'moab', 'sge', 'lsf', 'slurm', 'oar', 'kube').
{
address?: '127.0.0.1:8786', # The address of a Dask scheduler
timeout?: 5, # Timeout duration for initial connection to the scheduler
Expand Down Expand Up @@ -158,13 +173,33 @@ def execute(self, pipeline_context, execution_plan):
from dask_jobqueue import PBSCluster

cluster = PBSCluster(**self.build_dict(pipeline_name))
elif cluster_type == 'moab':
from dask_jobqueue import MoabCluster

cluster = MoabCluster(**self.build_dict(pipeline_name))
elif cluster_type == 'sge':
from dask_jobqueue import SGECluster

cluster = SGECluster(**self.build_dict(pipeline_name))
elif cluster_type == 'lsf':
from dask_jobqueue import LSFCluster

cluster = LSFCluster(**self.build_dict(pipeline_name))
elif cluster_type == 'slurm':
from dask_jobqueue import SLURMCluster

cluster = SLURMCluster(**self.build_dict(pipeline_name))
elif cluster_type == 'oar':
from dask_jobqueue import OARCluster

cluster = OARCluster(**self.build_dict(pipeline_name))
elif cluster_type == 'kube':
from dask_kubernetes import KubeCluster

cluster = KubeCluster(**self.build_dict(pipeline_name))
else:
raise ValueError(
f"Must be providing one of the following ('local', 'yarn', 'ssh', 'pbs', 'kube') not {cluster_type}"
f"Must be providing one of the following ('local', 'yarn', 'ssh', 'pbs', 'moab', 'sge', 'lsf', 'slurm', 'oar', 'kube') not {cluster_type}"
)

with dask.distributed.Client(cluster) as client:
Expand Down Expand Up @@ -232,8 +267,7 @@ def build_dict(self, pipeline_name):
<< use client here >>

'''

if self.cluster_type in ['yarn', 'kube', 'pbs']:
if self.cluster_type in ['yarn', 'pbs', 'moab', 'sge', 'lsf', 'slurm', 'oar', 'kube']:
dask_cfg = {'name': pipeline_name}
else:
dask_cfg = {}
Expand Down