Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

LSFCluster may be overly specific? #328

Closed
mrocklin opened this issue Aug 25, 2019 · 18 comments
Closed

LSFCluster may be overly specific? #328

mrocklin opened this issue Aug 25, 2019 · 18 comments

Comments

@mrocklin
Copy link
Member

I was trying out dask-jobqueue on the Summit supercomputer at Oak Ridge National Labs. I ran into a number of problems with our current configuration that seem to be special cases. I propose un-special-casing these, but I would like to get feedback from others who use LSF.

bsub command

We currently do some odd things with bsub. This made things fail for me on a login node (although they did work on a compute node). Removing this special-cased behavior made things work well for me in both places.

diff --git a/dask_jobqueue/lsf.py b/dask_jobqueue/lsf.py
index 95042a8..12126ee 100644
--- a/dask_jobqueue/lsf.py
+++ b/dask_jobqueue/lsf.py
@@ -53,7 +53,7 @@ class LSFJob(Job):
     """,
         4,
     )
-    submit_command = "bsub <"
+    submit_command = "bsub"
     cancel_command = "bkill"

     def __init__(
@@ -134,10 +134,6 @@ class LSFJob(Job):

         logger.debug("Job script: \n %s" % self.job_script())

-    def _submit_job(self, script_filename):
-        piped_cmd = [self.submit_command + " " + script_filename + " 2> /dev/null"]
-        return self._call(piped_cmd, shell=True)

cc'ing @raybellwaves and @guillaumeeb , who show up under git blame for this code

-R "span[hosts=1]"

My particular deployment didn't like these lines. I don't know if these are very useful generally though, and I can work around them.

            if ncpus > 1:
                # span[hosts=1] _might_ affect queue waiting
                # time, and is not required if ncpus==1
                header_lines.append('#BSUB -R "span[hosts=1]"')

cc @louisabraham (also pointed to by git blame)

@raybellwaves
Copy link
Member

Not at UM anymore. Pinging a colleague who may use this or know someone who can give feedback @milancurcic

@louisabraham
Copy link
Contributor

Hi,

-    submit_command = "bsub <"
+    submit_command = "bsub"

I know bsub can read the command from the args (not sure about stdin). Is this what this is about?

-R "span[hosts=1]"

I can confirm that this is necessary to allocate all processes on the same node. I have run into situations (with manually scheduled jobs on 32 cores) where not giving this argument resulted in the cores getting divided between multiple machines (like 16+16). For smaller numbers of cores, it might not be a problem though as LSF probably won't kill your job aggressively, but for the principle you should declare what you are actually using.

@lesteve
Copy link
Member

lesteve commented Aug 26, 2019

We currently do some odd things with bsub. This made things fail for me on a login node (although they did work on a compute node). Removing this special-cased behavior made things work well for me in both places.

This is weird and from what I remember from stdin was needed in @raybellwaves case (#78). Edit: I found this: #78 (comment)

@guillaumeeb
Copy link
Member

@mrocklin, what is you LSF version?

Its pretty certain you've got to use bsub < jobscript with LSF before. You can find it here or there

But I can find it in newer IBM docs.

For -R "span[hosts=1]", this is still documented.

@mrocklin
Copy link
Member Author

10.1.0.0, Jul 30 2019

@guillaumeeb
Copy link
Member

So this is the last one, maybe they changed the job script behaviour with this one, do you have access to IBM support through Summit staff?

The non working -R is weird though.

@louisabraham
Copy link
Contributor

I'm using IBM Spectrum LSF 10.1.0.0 build 492718, May 25 2018 and -R "span[hosts=1]" works flawlessly

@guillaumeeb
Copy link
Member

@louisabraham do you use bsub < job_script or bsub job_script?

@louisabraham
Copy link
Contributor

@guillaumeeb bsub job_script

@lesteve
Copy link
Member

lesteve commented Aug 27, 2019

@louisabraham I am a bit confused:

  • are you using dask_jobqueue.LSFCluster on your cluster right now? It seems to me that dask-jobqueue.LSFCLuster is using bsub < job_script.
  • does both bsub < job_script and bsub job_script work on your cluster?

In an ideal world we would figure out a way to make LSFCluster work both for older version of LSF and the newer versions of LSF.

@louisabraham
Copy link
Contributor

louisabraham commented Aug 27, 2019

@lesteve Sorry for not being clear, I'm not currently using dask_jobqueue.LSFCluster but I use bsub job_script daily. Do you want me to test dask_jobqueue.LSFCluster?

@lesteve
Copy link
Member

lesteve commented Aug 27, 2019

Do you want me to test dask_jobqueue.LSFCluster?

Can you first try to post the error you get if you do: bsub < job_script.

If you can give dask_jobqueue.LSFCluster a quick go, it would be useful too. Something like this should be good enough:

import time

from dask.distributed import Client

from dask_jobqueue import LSFCluster

# you may need additional arguments like queue and possibly others in `LSFCluster`
# look at cluster.job_script(), this is the script that is actually used with bsub
# and LSFCluster docstring
cluster = LSFCluster(cores=1, memory='1GB')
cluster.scale(1)

client = Client(cluster)

while len(client.scheduler_info()['workers']) < 1:
    print('waiting for workers')
    time.sleep(5)

fut = client.submit(lambda x: x, 1)
result = fut.result()
print('Got result:', result)

I am expecting is that you'll get an error at the cluster.scale line saying that the submit_command returned with a non-zero exit code.

If you have any feed-back about why you stopped using dask-jobqueue on your cluster, I would be curious to hear more about it!

@SimonBoothroyd
Copy link

Its pretty certain you've got to use bsub < jobscript with LSF before. You can find it here or there

On the cluster we use jobqueue on we seem to have to use bsub < jobscript - using just bsub jobscript results in the jobscript being ignored.

@stuarteberg
Copy link
Member

I'm also having trouble with this change:

-    submit_command = "bsub <"
+    submit_command = "bsub"

LSF outputs the following error:

/groups/flyem/home/bergs/.lsbatch/1571082744.66049265: line 8: /tmp/tmpggve1epx.sh: No such file or directory

So, /tmp/tmpggve1epx.sh is the submission script generated by dask_jobqueue/lsf.py, but once it gets sent to an LSF host (worker node), that node can't find the script. Of course that's no surprise -- it was written to a location (/tmp) that isn't on the shared file system.

If dask-jobqueue uses the old submit command (bsub <), then the entire script is "spooled", so there's no need for the target machine to have access to the file.

Thankfully, I notice that there is a new configuration option available, introduced in #347:

jobqueue:
  lsf:
    use-stdin: true

Using that setting fixes the issue for me.

BTW, I notice that use-stdin defaults to True IFF you're using (LSF < 10). I didn't track down the reasoning behind that, but in my case I'm using LSF 10.1.0.8 and use-stdin is still necessary for me. I wonder if that configuration option should default to true?

FWIW, here's my LSF version:

$ lsid | head -n1
IBM Spectrum LSF Standard 10.1.0.8, May 10 2019

@lesteve
Copy link
Member

lesteve commented Oct 15, 2019

@stuarteberg this is very useful feed-back, thanks a lot!

If dask-jobqueue uses the old submit command (bsub <), then the entire script is "spooled", so there's no need for the target machine to have access to the file.

Do you know if for LSF 10, there is an alternative way to spool the script that is different from bsub <. I found this in the LSF 10 doc that seems to indicate that bsub -is or bsub -i might do it but if you could tell us a command that works on your cluster it would be great.

BTW, I notice that use-stdin defaults to True IFF you're using (LSF < 10). I didn't track down the reasoning behind that, but in my case I'm using LSF 10.1.0.8 and use-stdin is still necessary for me. I wonder if that configuration option should default to true?

The main reason was @mrocklin's feed-back that bsub < was not working for him for LSF 10 and and the fact that bsub < was nowhere to be found in the LSF 10 doc (if someone finds bsub < in the LSF 10 doc let me know). I seem to remember @mrocklin pasted the output of the error somewhere but I was not able not find it ... @mrocklin if you could paste the error you have on Summit when using bsub < job_script this would be great.

Some additional general comments:

  • even for a given scheduler, there seems to be a lot of site-specific configuration (i.e. Oak Ridge Summit vs @stuarteberg LSF cluster) and we need user's feed-back to try to find what the most reasonable defaults are.
  • even in the ideal world where there was no site-specific configuration difference there is no way that dask-jobqueue maintainers can know all the cluster-specific quirks and none of the maintainers has access to all the kind of clusters we support. I am starting to think that we should try to build some kind of list of cluster-specific "referents" that are fine with being pinged for cluster-specific issues from time to time. @stuarteberg or @SimonBoothroyd if you would be fine to be on this list, let me know!
  • if people care about using dask-jobqueue on LSF and want the same level of support as for others schedulers (SLURM, PBS, SGE) it would be great if we could get some help on Add LSF docker files to CI  #115 to add LSF to our CI.

@stuarteberg
Copy link
Member

stuarteberg commented Oct 15, 2019

I found this in the LSF 10 doc that seems to indicate that bsub -is or bsub -i might do it

OK, I tried that. It turns out that option has a different meaning. bsub < mycommand.sh takes the command from standard in, but bsub -i data.txt mycommand.sh redirects data.txt as standard input to command.sh.

For example, the following two lines are equivalent:

bsub "sort < foo.txt > foo-sorted.txt"
bsub -i foo.txt "sort > foo-sorted.txt"

...except that in the latter case, foo.txt need not be accessible from the worker node. LSF will copy it if necessary.

The two options are orthogonal: I think all of the following are equivalent, assuming foo.txt is available on the shared filesystem:

bsub -i foo.txt ./command.sh
bsub -i foo.txt < command.sh
bsub "./command.sh < foo.txt"

the fact that bsub < was nowhere to be found in the LSF 10 doc

Good point, I was looking at the LSF 9 docs. In the LSF 10 docs, it's harder to find. But the bsub manpage does show that syntax:

https://www.ibm.com/support/knowledgecenter/en/SSWRJV_10.1.0/lsf_command_ref/bsub.zz1argument.command.1.html

@lesteve
Copy link
Member

lesteve commented Oct 16, 2019

Thanks for your great feed-back. I guess I was too optimistic in thinking that "spool" was meaning the same thing in one of your earlier message and in the LSF docs. Full disclosure I had no idea what spool means (and I don't have the bandwidth to investigate).

At this point, I am starting to think that Oak Ridge Summit may be the odd one out since bsub < job_script is mentioned in the manpage for LSF 10 ... and that maybe use_stdin should be True by default. As with other issues about LSF in dask-jobqueue, it would be great to have people with LSF sys-admin skills involved. It could potentially save us a lot of time.

Here are a few action points:

  • find people around you that have access to a LSFCluster and ask them nicely the output of lsid | head -n1 and whether bsub < job_script works for them so we chose a more appropriate default for use_stdin
  • @mrocklin if you could paste the output of this snippet on Summit. If there is an error it'd be great if you could ask Summit sys-admin if they have insights into why.
from dask_jobqueue import LSFCluster
cluster = LSFCluster(
    cores=128,
    memory="600 GB",
    project="GEN119",
    walltime="00:30",
    use_stdin=True,
)
cluster.scale(jobs=3)  # ask for three nodes

Side-comment: for OARCluster I had a similar issue (the job script needs to exist when the job starts so the approach we have of generating a temporary file for the job script and then deleting it immediately after submission did not work for OAR either). The hack I used is some hacky parsing (see

def _submit_job(self, fn):
# OAR specificity: the submission script needs to exist on the worker
# when the job starts on the worker. This is different from other
# schedulers that only need the script on the submission node at
# submission time. That means that we can not use the same strategy as
# in JobQueueCluster: create a temporary submission file, submit the
# script, delete the submission file. In order to reuse the code in
# the base JobQueueCluster class, we read from the temporary file and
# reconstruct the command line where the script is passed in as a
# string (inline script in OAR jargon) rather than as a filename.
with open(fn) as f:
content_lines = f.readlines()
oar_lines = [line for line in content_lines if line.startswith("#OAR ")]
oarsub_options = [line.replace("#OAR ", "").strip() for line in oar_lines]
inline_script_lines = [
line for line in content_lines if not line.startswith("#")
]
inline_script = "".join(inline_script_lines)
oarsub_command = " ".join([self.submit_command] + oarsub_options)
oarsub_command_split = shlex.split(oarsub_command) + [inline_script]
return self._call(oarsub_command_split)
) of the job script to turn the script header into command arguments. I thought OAR was the only job scheduler with this quirk but apparently LSF may want to join the party ...

@guillaumeeb
Copy link
Member

I'm going to close this one in favor of #372 and #338.
Thanks everyone for all the discusssion here, and feel free to re-open the issue if you think there are still things to be done.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

No branches or pull requests

7 participants