// Dispatcher service for Crunch that submits containers to the slurm queue.
import (
+ "bytes"
"context"
"flag"
"fmt"
"strings"
"time"
+ "git.curoverse.com/arvados.git/lib/dispatchcloud"
"git.curoverse.com/arvados.git/sdk/go/arvados"
"git.curoverse.com/arvados.git/sdk/go/arvadosclient"
"git.curoverse.com/arvados.git/sdk/go/config"
"git.curoverse.com/arvados.git/sdk/go/dispatch"
- "git.curoverse.com/arvados.git/services/dispatchcloud"
"github.com/coreos/go-systemd/daemon"
+ "github.com/sirupsen/logrus"
)
-var version = "dev"
+type logger interface {
+ dispatch.Logger
+ Fatalf(string, ...interface{})
+}
+
+const initialNiceValue int64 = 10000
+
+var (
+ version = "dev"
+ defaultConfigPath = "/etc/arvados/crunch-dispatch-slurm/crunch-dispatch-slurm.yml"
+)
-type command struct {
- dispatcher *dispatch.Dispatcher
- cluster *arvados.Cluster
- sqCheck *SqueueChecker
- slurm Slurm
+type Dispatcher struct {
+ *dispatch.Dispatcher
+ logger logrus.FieldLogger
+ cluster *arvados.Cluster
+ sqCheck *SqueueChecker
+ slurm Slurm
Client arvados.Client
SbatchArguments []string
PollPeriod arvados.Duration
+ PrioritySpread int64
// crunch-run command to invoke. The container UUID will be
// appended. If nil, []string{"crunch-run"} will be used.
// Example: []string{"crunch-run", "--cgroup-parent-subsystem=memory"}
CrunchRunCommand []string
+ // Extra RAM to reserve (in Bytes) for SLURM job, in addition
+ // to the amount specified in the container's RuntimeConstraints
+ ReserveExtraRAM int64
+
// Minimum time between two attempts to run the same container
MinRetryPeriod arvados.Duration
+
+ // Batch size for container queries
+ BatchSize int64
}
func main() {
- err := (&command{}).Run(os.Args[0], os.Args[1:])
+ logger := logrus.StandardLogger()
+ if os.Getenv("DEBUG") != "" {
+ logger.SetLevel(logrus.DebugLevel)
+ }
+ logger.Formatter = &logrus.JSONFormatter{
+ TimestampFormat: "2006-01-02T15:04:05.000000000Z07:00",
+ }
+ disp := &Dispatcher{logger: logger}
+ err := disp.Run(os.Args[0], os.Args[1:])
if err != nil {
- log.Fatal(err)
+ logrus.Fatalf("%s", err)
}
}
-const defaultConfigPath = "/etc/arvados/crunch-dispatch-slurm/crunch-dispatch-slurm.yml"
+func (disp *Dispatcher) Run(prog string, args []string) error {
+ if err := disp.configure(prog, args); err != nil {
+ return err
+ }
+ disp.setup()
+ return disp.run()
+}
-func (cmd *command) Run(prog string, args []string) error {
+// configure() loads config files. Tests skip this.
+func (disp *Dispatcher) configure(prog string, args []string) error {
flags := flag.NewFlagSet(prog, flag.ExitOnError)
flags.Usage = func() { usage(flags) }
return nil
}
- log.Printf("crunch-dispatch-slurm %s started", version)
+ disp.logger.Printf("crunch-dispatch-slurm %s started", version)
- err := cmd.readConfig(*configPath)
+ err := disp.readConfig(*configPath)
if err != nil {
return err
}
- if cmd.CrunchRunCommand == nil {
- cmd.CrunchRunCommand = []string{"crunch-run"}
+ if disp.CrunchRunCommand == nil {
+ disp.CrunchRunCommand = []string{"crunch-run"}
}
- if cmd.PollPeriod == 0 {
- cmd.PollPeriod = arvados.Duration(10 * time.Second)
+ if disp.PollPeriod == 0 {
+ disp.PollPeriod = arvados.Duration(10 * time.Second)
}
- if cmd.Client.APIHost != "" || cmd.Client.AuthToken != "" {
+ if disp.Client.APIHost != "" || disp.Client.AuthToken != "" {
// Copy real configs into env vars so [a]
// MakeArvadosClient() uses them, and [b] they get
// propagated to crunch-run via SLURM.
- os.Setenv("ARVADOS_API_HOST", cmd.Client.APIHost)
- os.Setenv("ARVADOS_API_TOKEN", cmd.Client.AuthToken)
+ os.Setenv("ARVADOS_API_HOST", disp.Client.APIHost)
+ os.Setenv("ARVADOS_API_TOKEN", disp.Client.AuthToken)
os.Setenv("ARVADOS_API_HOST_INSECURE", "")
- if cmd.Client.Insecure {
+ if disp.Client.Insecure {
os.Setenv("ARVADOS_API_HOST_INSECURE", "1")
}
- os.Setenv("ARVADOS_KEEP_SERVICES", strings.Join(cmd.Client.KeepServiceURIs, " "))
+ os.Setenv("ARVADOS_KEEP_SERVICES", strings.Join(disp.Client.KeepServiceURIs, " "))
os.Setenv("ARVADOS_EXTERNAL_CLIENT", "")
} else {
- log.Printf("warning: Client credentials missing from config, so falling back on environment variables (deprecated).")
+ disp.logger.Warnf("Client credentials missing from config, so falling back on environment variables (deprecated).")
}
if *dumpConfig {
- log.Fatal(config.DumpAndExit(cmd))
+ return config.DumpAndExit(disp)
}
- arv, err := arvadosclient.MakeArvadosClient()
- if err != nil {
- log.Printf("Error making Arvados client: %v", err)
- return err
- }
- arv.Retries = 25
-
siteConfig, err := arvados.GetConfig(arvados.DefaultConfigFile)
if os.IsNotExist(err) {
- log.Printf("warning: no cluster config file %q (%s), proceeding with no node types defined", arvados.DefaultConfigFile, err)
+ disp.logger.Warnf("no cluster config (%s), proceeding with no node types defined", err)
} else if err != nil {
- log.Fatalf("error loading config: %s", err)
- } else if cmd.cluster, err = siteConfig.GetCluster(""); err != nil {
- log.Fatalf("config error: %s", err)
- } else if len(cmd.cluster.InstanceTypes) > 0 {
- go dispatchcloud.SlurmNodeTypeFeatureKludge(cmd.cluster)
+ return fmt.Errorf("error loading config: %s", err)
+ } else if disp.cluster, err = siteConfig.GetCluster(""); err != nil {
+ return fmt.Errorf("config error: %s", err)
}
- if cmd.slurm == nil {
- cmd.slurm = &slurmCLI{}
- }
+ return nil
+}
- cmd.sqCheck = &SqueueChecker{
- Period: time.Duration(cmd.PollPeriod),
- Slurm: cmd.slurm,
+// setup() initializes private fields after configure().
+func (disp *Dispatcher) setup() {
+ if disp.logger == nil {
+ disp.logger = logrus.StandardLogger()
}
- defer cmd.sqCheck.Stop()
+ arv, err := arvadosclient.MakeArvadosClient()
+ if err != nil {
+ disp.logger.Fatalf("Error making Arvados client: %v", err)
+ }
+ arv.Retries = 25
- cmd.dispatcher = &dispatch.Dispatcher{
+ disp.slurm = NewSlurmCLI()
+ disp.sqCheck = &SqueueChecker{
+ Logger: disp.logger,
+ Period: time.Duration(disp.PollPeriod),
+ PrioritySpread: disp.PrioritySpread,
+ Slurm: disp.slurm,
+ }
+ disp.Dispatcher = &dispatch.Dispatcher{
Arv: arv,
- RunContainer: cmd.run,
- PollPeriod: time.Duration(cmd.PollPeriod),
- MinRetryPeriod: time.Duration(cmd.MinRetryPeriod),
+ Logger: disp.logger,
+ BatchSize: disp.BatchSize,
+ RunContainer: disp.runContainer,
+ PollPeriod: time.Duration(disp.PollPeriod),
+ MinRetryPeriod: time.Duration(disp.MinRetryPeriod),
+ }
+}
+
+func (disp *Dispatcher) run() error {
+ defer disp.sqCheck.Stop()
+
+ if disp.cluster != nil && len(disp.cluster.InstanceTypes) > 0 {
+ go SlurmNodeTypeFeatureKludge(disp.cluster)
}
if _, err := daemon.SdNotify(false, "READY=1"); err != nil {
log.Printf("Error notifying init daemon: %v", err)
}
-
- go cmd.checkSqueueForOrphans()
-
- return cmd.dispatcher.Run(context.Background())
+ go disp.checkSqueueForOrphans()
+ return disp.Dispatcher.Run(context.Background())
}
var containerUuidPattern = regexp.MustCompile(`^[a-z0-9]{5}-dz642-[a-z0-9]{15}$`)
// jobs started by a previous dispatch process that never released
// their slurm allocations even though their container states are
// Cancelled or Complete. See https://dev.arvados.org/issues/10979
-func (cmd *command) checkSqueueForOrphans() {
- for _, uuid := range cmd.sqCheck.All() {
+func (disp *Dispatcher) checkSqueueForOrphans() {
+ for _, uuid := range disp.sqCheck.All() {
if !containerUuidPattern.MatchString(uuid) {
continue
}
- err := cmd.dispatcher.TrackContainer(uuid)
+ err := disp.TrackContainer(uuid)
if err != nil {
log.Printf("checkSqueueForOrphans: TrackContainer(%s): %s", uuid, err)
}
}
}
-func (cmd *command) niceness(priority int) int {
- if priority > 1000 {
- priority = 1000
- }
- if priority < 0 {
- priority = 0
- }
- // Niceness range 1-10000
- return (1000 - priority) * 10
-}
-
-func (cmd *command) sbatchArgs(container arvados.Container) ([]string, error) {
- mem := int64(math.Ceil(float64(container.RuntimeConstraints.RAM+container.RuntimeConstraints.KeepCacheRAM) / float64(1048576)))
+func (disp *Dispatcher) slurmConstraintArgs(container arvados.Container) []string {
+ mem := int64(math.Ceil(float64(container.RuntimeConstraints.RAM+container.RuntimeConstraints.KeepCacheRAM+disp.ReserveExtraRAM) / float64(1048576)))
- var disk int64
- for _, m := range container.Mounts {
- if m.Kind == "tmp" {
- disk += m.Capacity
- }
- }
+ disk := dispatchcloud.EstimateScratchSpace(&container)
disk = int64(math.Ceil(float64(disk) / float64(1048576)))
-
- var sbatchArgs []string
- sbatchArgs = append(sbatchArgs, cmd.SbatchArguments...)
- sbatchArgs = append(sbatchArgs, fmt.Sprintf("--job-name=%s", container.UUID))
- sbatchArgs = append(sbatchArgs, fmt.Sprintf("--mem=%d", mem))
- sbatchArgs = append(sbatchArgs, fmt.Sprintf("--cpus-per-task=%d", container.RuntimeConstraints.VCPUs))
- sbatchArgs = append(sbatchArgs, fmt.Sprintf("--tmp=%d", disk))
- sbatchArgs = append(sbatchArgs, fmt.Sprintf("--nice=%d", cmd.niceness(container.Priority)))
- if len(container.SchedulingParameters.Partitions) > 0 {
- sbatchArgs = append(sbatchArgs, fmt.Sprintf("--partition=%s", strings.Join(container.SchedulingParameters.Partitions, ",")))
+ return []string{
+ fmt.Sprintf("--mem=%d", mem),
+ fmt.Sprintf("--cpus-per-task=%d", container.RuntimeConstraints.VCPUs),
+ fmt.Sprintf("--tmp=%d", disk),
}
+}
+
+func (disp *Dispatcher) sbatchArgs(container arvados.Container) ([]string, error) {
+ var args []string
+ args = append(args, disp.SbatchArguments...)
+ args = append(args, "--job-name="+container.UUID, fmt.Sprintf("--nice=%d", initialNiceValue), "--no-requeue")
- if cmd.cluster == nil {
+ if disp.cluster == nil {
// no instance types configured
- } else if it, err := dispatchcloud.ChooseInstanceType(cmd.cluster, &container); err == dispatchcloud.ErrInstanceTypesNotConfigured {
+ args = append(args, disp.slurmConstraintArgs(container)...)
+ } else if it, err := dispatchcloud.ChooseInstanceType(disp.cluster, &container); err == dispatchcloud.ErrInstanceTypesNotConfigured {
// ditto
+ args = append(args, disp.slurmConstraintArgs(container)...)
} else if err != nil {
return nil, err
} else {
- sbatchArgs = append(sbatchArgs, "--constraint="+it.Name)
+ // use instancetype constraint instead of slurm mem/cpu/tmp specs
+ args = append(args, "--constraint=instancetype="+it.Name)
}
- return sbatchArgs, nil
+ if len(container.SchedulingParameters.Partitions) > 0 {
+ args = append(args, "--partition="+strings.Join(container.SchedulingParameters.Partitions, ","))
+ }
+
+ return args, nil
}
-func (cmd *command) submit(container arvados.Container, crunchRunCommand []string) error {
+func (disp *Dispatcher) submit(container arvados.Container, crunchRunCommand []string) error {
// append() here avoids modifying crunchRunCommand's
// underlying array, which is shared with other goroutines.
crArgs := append([]string(nil), crunchRunCommand...)
crArgs = append(crArgs, container.UUID)
crScript := strings.NewReader(execScript(crArgs))
- cmd.sqCheck.L.Lock()
- defer cmd.sqCheck.L.Unlock()
-
- sbArgs, err := cmd.sbatchArgs(container)
+ sbArgs, err := disp.sbatchArgs(container)
if err != nil {
return err
}
log.Printf("running sbatch %+q", sbArgs)
- return cmd.slurm.Batch(crScript, sbArgs)
+ return disp.slurm.Batch(crScript, sbArgs)
}
// Submit a container to the slurm queue (or resume monitoring if it's
// already in the queue). Cancel the slurm job if the container's
// priority changes to zero or its state indicates it's no longer
// running.
-func (cmd *command) run(_ *dispatch.Dispatcher, ctr arvados.Container, status <-chan arvados.Container) {
+func (disp *Dispatcher) runContainer(_ *dispatch.Dispatcher, ctr arvados.Container, status <-chan arvados.Container) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- if ctr.State == dispatch.Locked && !cmd.sqCheck.HasUUID(ctr.UUID) {
+ if ctr.State == dispatch.Locked && !disp.sqCheck.HasUUID(ctr.UUID) {
log.Printf("Submitting container %s to slurm", ctr.UUID)
- if err := cmd.submit(ctr, cmd.CrunchRunCommand); err != nil {
+ if err := disp.submit(ctr, disp.CrunchRunCommand); err != nil {
var text string
- if err == dispatchcloud.ErrConstraintsNotSatisfiable {
- text = fmt.Sprintf("cannot run container %s: %s", ctr.UUID, err)
- cmd.dispatcher.UpdateState(ctr.UUID, dispatch.Cancelled)
+ if err, ok := err.(dispatchcloud.ConstraintsNotSatisfiableError); ok {
+ var logBuf bytes.Buffer
+ fmt.Fprintf(&logBuf, "cannot run container %s: %s\n", ctr.UUID, err)
+ if len(err.AvailableTypes) == 0 {
+ fmt.Fprint(&logBuf, "No instance types are configured.\n")
+ } else {
+ fmt.Fprint(&logBuf, "Available instance types:\n")
+ for _, t := range err.AvailableTypes {
+ fmt.Fprintf(&logBuf,
+ "Type %q: %d VCPUs, %d RAM, %d Scratch, %f Price\n",
+ t.Name, t.VCPUs, t.RAM, t.Scratch, t.Price,
+ )
+ }
+ }
+ text = logBuf.String()
+ disp.UpdateState(ctr.UUID, dispatch.Cancelled)
} else {
text = fmt.Sprintf("Error submitting container %s to slurm: %s", ctr.UUID, err)
}
"object_uuid": ctr.UUID,
"event_type": "dispatch",
"properties": map[string]string{"text": text}}}
- cmd.dispatcher.Arv.Create("logs", lr, nil)
+ disp.Arv.Create("logs", lr, nil)
- cmd.dispatcher.Unlock(ctr.UUID)
+ disp.Unlock(ctr.UUID)
return
}
}
// no point in waiting for further dispatch updates: just
// clean up and return.
go func(uuid string) {
- for ctx.Err() == nil && cmd.sqCheck.HasUUID(uuid) {
+ for ctx.Err() == nil && disp.sqCheck.HasUUID(uuid) {
}
cancel()
}(ctr.UUID)
select {
case <-ctx.Done():
// Disappeared from squeue
- if err := cmd.dispatcher.Arv.Get("containers", ctr.UUID, nil, &ctr); err != nil {
- log.Printf("Error getting final container state for %s: %s", ctr.UUID, err)
+ if err := disp.Arv.Get("containers", ctr.UUID, nil, &ctr); err != nil {
+ log.Printf("error getting final container state for %s: %s", ctr.UUID, err)
}
switch ctr.State {
case dispatch.Running:
- cmd.dispatcher.UpdateState(ctr.UUID, dispatch.Cancelled)
+ disp.UpdateState(ctr.UUID, dispatch.Cancelled)
case dispatch.Locked:
- cmd.dispatcher.Unlock(ctr.UUID)
+ disp.Unlock(ctr.UUID)
}
return
case updated, ok := <-status:
if !ok {
- log.Printf("Dispatcher says container %s is done: cancel slurm job", ctr.UUID)
- cmd.scancel(ctr)
+ log.Printf("container %s is done: cancel slurm job", ctr.UUID)
+ disp.scancel(ctr)
} else if updated.Priority == 0 {
- log.Printf("Container %s has state %q, priority %d: cancel slurm job", ctr.UUID, updated.State, updated.Priority)
- cmd.scancel(ctr)
+ log.Printf("container %s has state %q, priority %d: cancel slurm job", ctr.UUID, updated.State, updated.Priority)
+ disp.scancel(ctr)
} else {
- cmd.renice(updated)
+ p := int64(updated.Priority)
+ if p <= 1000 {
+ // API is providing
+ // user-assigned priority. If
+ // ctrs have equal priority,
+ // run the older one first.
+ p = int64(p)<<50 - (updated.CreatedAt.UnixNano() >> 14)
+ }
+ disp.sqCheck.SetPriority(ctr.UUID, p)
}
}
}
}
-
-func (cmd *command) scancel(ctr arvados.Container) {
- cmd.sqCheck.L.Lock()
- err := cmd.slurm.Cancel(ctr.UUID)
- cmd.sqCheck.L.Unlock()
-
+func (disp *Dispatcher) scancel(ctr arvados.Container) {
+ err := disp.slurm.Cancel(ctr.UUID)
if err != nil {
log.Printf("scancel: %s", err)
time.Sleep(time.Second)
- } else if cmd.sqCheck.HasUUID(ctr.UUID) {
+ } else if disp.sqCheck.HasUUID(ctr.UUID) {
log.Printf("container %s is still in squeue after scancel", ctr.UUID)
time.Sleep(time.Second)
}
}
-func (cmd *command) renice(ctr arvados.Container) {
- nice := cmd.niceness(ctr.Priority)
- oldnice := cmd.sqCheck.GetNiceness(ctr.UUID)
- if nice == oldnice || oldnice == -1 {
- return
- }
- log.Printf("updating slurm nice value to %d (was %d)", nice, oldnice)
- cmd.sqCheck.L.Lock()
- err := cmd.slurm.Renice(ctr.UUID, nice)
- cmd.sqCheck.L.Unlock()
-
- if err != nil {
- log.Printf("renice: %s", err)
- time.Sleep(time.Second)
- return
- }
- if cmd.sqCheck.HasUUID(ctr.UUID) {
- log.Printf("container %s has arvados priority %d, slurm nice %d",
- ctr.UUID, ctr.Priority, cmd.sqCheck.GetNiceness(ctr.UUID))
- }
-}
-
-func (cmd *command) readConfig(path string) error {
- err := config.LoadFile(cmd, path)
+func (disp *Dispatcher) readConfig(path string) error {
+ err := config.LoadFile(disp, path)
if err != nil && os.IsNotExist(err) && path == defaultConfigPath {
log.Printf("Config not specified. Continue with default configuration.")
err = nil