More cleanup, remove references to Ubuntu 16.04 (xenial) which is no
[arvados.git] / services / crunch-dispatch-local / crunch-dispatch-local.go
index e05c0c5da4439e44931837ea5a259885624b80d8..a3cb1341a4677e7ecdc7c03976da7483e47c1aa5 100644 (file)
@@ -1,34 +1,54 @@
+// Copyright (C) The Arvados Authors. All rights reserved.
+//
+// SPDX-License-Identifier: AGPL-3.0
+
 package main
 
+// Dispatcher service for Crunch that runs containers locally.
+
 import (
+       "context"
        "flag"
-       "git.curoverse.com/arvados.git/sdk/go/arvadosclient"
-       "log"
+       "fmt"
        "os"
        "os/exec"
        "os/signal"
        "sync"
        "syscall"
        "time"
+
+       "git.arvados.org/arvados.git/lib/config"
+       "git.arvados.org/arvados.git/sdk/go/arvados"
+       "git.arvados.org/arvados.git/sdk/go/arvadosclient"
+       "git.arvados.org/arvados.git/sdk/go/dispatch"
+       "github.com/sirupsen/logrus"
 )
 
+var version = "dev"
+
 func main() {
        err := doMain()
        if err != nil {
-               log.Fatalf("%q", err)
+               logrus.Fatalf("%q", err)
        }
 }
 
 var (
-       arv              arvadosclient.ArvadosClient
        runningCmds      map[string]*exec.Cmd
        runningCmdsMutex sync.Mutex
        waitGroup        sync.WaitGroup
-       doneProcessing   chan bool
-       sigChan          chan os.Signal
+       crunchRunCommand *string
 )
 
 func doMain() error {
+       logger := logrus.StandardLogger()
+       if os.Getenv("DEBUG") != "" {
+               logger.SetLevel(logrus.DebugLevel)
+       }
+       logger.Formatter = &logrus.JSONFormatter{
+               TimestampFormat: "2006-01-02T15:04:05.000000000Z07:00",
+       }
+
        flags := flag.NewFlagSet("crunch-dispatch-local", flag.ExitOnError)
 
        pollInterval := flags.Int(
@@ -36,48 +56,91 @@ func doMain() error {
                10,
                "Interval in seconds to poll for queued containers")
 
-       priorityPollInterval := flags.Int(
-               "container-priority-poll-interval",
-               60,
-               "Interval in seconds to check priority of a dispatched container")
-
-       crunchRunCommand := flags.String(
+       crunchRunCommand = flags.String(
                "crunch-run-command",
                "/usr/bin/crunch-run",
                "Crunch command to run container")
 
+       getVersion := flags.Bool(
+               "version",
+               false,
+               "Print version information and exit.")
+
        // Parse args; omit the first arg which is the command name
        flags.Parse(os.Args[1:])
 
-       var err error
-       arv, err = arvadosclient.MakeArvadosClient()
+       // Print version information if requested
+       if *getVersion {
+               fmt.Printf("crunch-dispatch-local %s\n", version)
+               return nil
+       }
+
+       loader := config.NewLoader(nil, logger)
+       cfg, err := loader.Load()
+       cluster, err := cfg.GetCluster("")
        if err != nil {
-               return err
+               return fmt.Errorf("config error: %s", err)
        }
 
-       // Channel to terminate
-       doneProcessing = make(chan bool)
+       logger.Printf("crunch-dispatch-local %s started", version)
 
-       // Map of running crunch jobs
        runningCmds = make(map[string]*exec.Cmd)
 
-       // Graceful shutdown
-       sigChan = make(chan os.Signal, 1)
-       signal.Notify(sigChan, syscall.SIGINT, syscall.SIGTERM, syscall.SIGQUIT)
-       go func(sig <-chan os.Signal) {
-               for sig := range sig {
-                       log.Printf("Caught signal: %v", sig)
-                       doneProcessing <- true
+       var client arvados.Client
+       client.APIHost = cluster.Services.Controller.ExternalURL.Host
+       client.AuthToken = cluster.SystemRootToken
+       client.Insecure = cluster.TLS.Insecure
+
+       if client.APIHost != "" || 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", client.APIHost)
+               os.Setenv("ARVADOS_API_TOKEN", client.AuthToken)
+               os.Setenv("ARVADOS_API_HOST_INSECURE", "")
+               if client.Insecure {
+                       os.Setenv("ARVADOS_API_HOST_INSECURE", "1")
                }
-       }(sigChan)
+               os.Setenv("ARVADOS_EXTERNAL_CLIENT", "")
+       } else {
+               logger.Warnf("Client credentials missing from config, so falling back on environment variables (deprecated).")
+       }
 
-       // Run all queued containers
-       runQueuedContainers(*pollInterval, *priorityPollInterval, *crunchRunCommand)
+       arv, err := arvadosclient.MakeArvadosClient()
+       if err != nil {
+               logger.Errorf("error making Arvados client: %v", err)
+               return err
+       }
+       arv.Retries = 25
 
+       ctx, cancel := context.WithCancel(context.Background())
+
+       dispatcher := dispatch.Dispatcher{
+               Logger:       logger,
+               Arv:          arv,
+               RunContainer: (&LocalRun{startFunc, make(chan bool, 8), ctx, cluster}).run,
+               PollPeriod:   time.Duration(*pollInterval) * time.Second,
+       }
+
+       err = dispatcher.Run(ctx)
+       if err != nil {
+               return err
+       }
+
+       c := make(chan os.Signal, 1)
+       signal.Notify(c, os.Interrupt, syscall.SIGTERM, syscall.SIGQUIT)
+       sig := <-c
+       logger.Printf("Received %s, shutting down", sig)
+       signal.Stop(c)
+
+       cancel()
+
+       runningCmdsMutex.Lock()
        // Finished dispatching; interrupt any crunch jobs that are still running
        for _, cmd := range runningCmds {
                cmd.Process.Signal(os.Interrupt)
        }
+       runningCmdsMutex.Unlock()
 
        // Wait for all running crunch jobs to complete / terminate
        waitGroup.Wait()
@@ -85,136 +148,127 @@ func doMain() error {
        return nil
 }
 
-// Poll for queued containers using pollInterval.
-// Invoke dispatchLocal for each ticker cycle, which will run all the queued containers.
-//
-// Any errors encountered are logged but the program would continue to run (not exit).
-// This is because, once one or more crunch jobs are running,
-// we would need to wait for them complete.
-func runQueuedContainers(pollInterval, priorityPollInterval int, crunchRunCommand string) {
-       ticker := time.NewTicker(time.Duration(pollInterval) * time.Second)
-
-       for {
-               select {
-               case <-ticker.C:
-                       dispatchLocal(priorityPollInterval, crunchRunCommand)
-               case <-doneProcessing:
-                       ticker.Stop()
-                       return
-               }
-       }
+func startFunc(container arvados.Container, cmd *exec.Cmd) error {
+       return cmd.Start()
 }
 
-// Container data
-type Container struct {
-       UUID     string `json:"uuid"`
-       State    string `json:"state"`
-       Priority int    `json:"priority"`
+type LocalRun struct {
+       startCmd         func(container arvados.Container, cmd *exec.Cmd) error
+       concurrencyLimit chan bool
+       ctx              context.Context
+       cluster          *arvados.Cluster
 }
 
-// ContainerList is a list of the containers from api
-type ContainerList struct {
-       Items []Container `json:"items"`
-}
+// Run a container.
+//
+// If the container is Locked, start a new crunch-run process and wait until
+// crunch-run completes.  If the priority is set to zero, set an interrupt
+// signal to the crunch-run process.
+//
+// If the container is in any other state, or is not Complete/Cancelled after
+// crunch-run terminates, mark the container as Cancelled.
+func (lr *LocalRun) run(dispatcher *dispatch.Dispatcher,
+       container arvados.Container,
+       status <-chan arvados.Container) error {
 
-// Get the list of queued containers from API server and invoke run for each container.
-func dispatchLocal(priorityPollInterval int, crunchRunCommand string) {
-       params := arvadosclient.Dict{
-               "filters": [][]string{[]string{"state", "=", "Queued"}},
-       }
+       uuid := container.UUID
 
-       var containers ContainerList
-       err := arv.List("containers", params, &containers)
-       if err != nil {
-               log.Printf("Error getting list of queued containers: %q", err)
-               return
-       }
+       if container.State == dispatch.Locked {
 
-       for i := 0; i < len(containers.Items); i++ {
-               log.Printf("About to run queued container %v", containers.Items[i].UUID)
-               // Run the container
-               go run(containers.Items[i].UUID, crunchRunCommand, priorityPollInterval)
-       }
-}
+               select {
+               case lr.concurrencyLimit <- true:
+                       break
+               case <-lr.ctx.Done():
+                       return lr.ctx.Err()
+               }
 
-// Run queued container:
-// Set container state to locked (TBD)
-// Run container using the given crunch-run command
-// Set the container state to Running
-// If the container priority becomes zero while crunch job is still running, terminate it.
-func run(uuid string, crunchRunCommand string, priorityPollInterval int) {
-       cmd := exec.Command(crunchRunCommand, uuid)
-
-       cmd.Stdin = nil
-       cmd.Stderr = os.Stderr
-       cmd.Stdout = os.Stderr
-       if err := cmd.Start(); err != nil {
-               log.Printf("Error running container for %v: %q", uuid, err)
-               return
-       }
+               defer func() { <-lr.concurrencyLimit }()
 
-       // Add this crunch job to the list of runningCmds
-       runningCmdsMutex.Lock()
-       runningCmds[uuid] = cmd
-       runningCmdsMutex.Unlock()
+               select {
+               case c := <-status:
+                       // Check for state updates after possibly
+                       // waiting to be ready-to-run
+                       if c.Priority == 0 {
+                               goto Finish
+                       }
+               default:
+                       break
+               }
 
-       log.Printf("Started container run for %v", uuid)
+               waitGroup.Add(1)
+               defer waitGroup.Done()
 
-       // Add this crunch job to waitGroup
-       waitGroup.Add(1)
-       defer waitGroup.Done()
+               cmd := exec.Command(*crunchRunCommand, "--runtime-engine="+lr.cluster.Containers.RuntimeEngine, uuid)
+               cmd.Stdin = nil
+               cmd.Stderr = os.Stderr
+               cmd.Stdout = os.Stderr
 
-       // Update container status to Running
-       err := arv.Update("containers", uuid,
-               arvadosclient.Dict{
-                       "container": arvadosclient.Dict{"state": "Running"}},
-               nil)
-       if err != nil {
-               log.Printf("Error updating container state to 'Running' for %v: %q", uuid, err)
-       }
+               dispatcher.Logger.Printf("starting container %v", uuid)
+
+               // Add this crunch job to the list of runningCmds only if we
+               // succeed in starting crunch-run.
+
+               runningCmdsMutex.Lock()
+               if err := lr.startCmd(container, cmd); err != nil {
+                       runningCmdsMutex.Unlock()
+                       dispatcher.Logger.Warnf("error starting %q for %s: %s", *crunchRunCommand, uuid, err)
+                       dispatcher.UpdateState(uuid, dispatch.Cancelled)
+               } else {
+                       runningCmds[uuid] = cmd
+                       runningCmdsMutex.Unlock()
+
+                       // Need to wait for crunch-run to exit
+                       done := make(chan struct{})
 
-       // A goroutine to terminate the runner if container priority becomes zero
-       priorityTicker := time.NewTicker(time.Duration(priorityPollInterval) * time.Second)
-       go func() {
-               for _ = range priorityTicker.C {
-                       var container Container
-                       err := arv.Get("containers", uuid, nil, &container)
-                       if err != nil {
-                               log.Printf("Error getting container info for %v: %q", uuid, err)
-                       } else {
-                               if container.Priority == 0 {
-                                       priorityTicker.Stop()
-                                       cmd.Process.Signal(os.Interrupt)
+                       go func() {
+                               if _, err := cmd.Process.Wait(); err != nil {
+                                       dispatcher.Logger.Warnf("error while waiting for crunch job to finish for %v: %q", uuid, err)
+                               }
+                               dispatcher.Logger.Debugf("sending done")
+                               done <- struct{}{}
+                       }()
+
+               Loop:
+                       for {
+                               select {
+                               case <-done:
+                                       break Loop
+                               case c := <-status:
+                                       // Interrupt the child process if priority changes to 0
+                                       if (c.State == dispatch.Locked || c.State == dispatch.Running) && c.Priority == 0 {
+                                               dispatcher.Logger.Printf("sending SIGINT to pid %d to cancel container %v", cmd.Process.Pid, uuid)
+                                               cmd.Process.Signal(os.Interrupt)
+                                       }
                                }
                        }
-               }
-       }()
+                       close(done)
 
-       // Wait for the crunch job to exit
-       if _, err := cmd.Process.Wait(); err != nil {
-               log.Printf("Error while waiting for crunch job to finish for %v: %q", uuid, err)
+                       dispatcher.Logger.Printf("finished container run for %v", uuid)
+
+                       // Remove the crunch job from runningCmds
+                       runningCmdsMutex.Lock()
+                       delete(runningCmds, uuid)
+                       runningCmdsMutex.Unlock()
+               }
        }
 
-       // Remove the crunch job to runningCmds
-       runningCmdsMutex.Lock()
-       delete(runningCmds, uuid)
-       runningCmdsMutex.Unlock()
+Finish:
 
-       priorityTicker.Stop()
-
-       // The container state should be 'Complete'
-       var container Container
-       err = arv.Get("containers", uuid, nil, &container)
-       if container.State == "Running" {
-               log.Printf("After crunch-run process termination, the state is still 'Running' for %v. Updating it to 'Complete'", uuid)
-               err = arv.Update("containers", uuid,
-                       arvadosclient.Dict{
-                               "container": arvadosclient.Dict{"state": "Complete"}},
-                       nil)
-               if err != nil {
-                       log.Printf("Error updating container state to Complete for %v: %q", uuid, err)
-               }
+       // If the container is not finalized, then change it to "Cancelled".
+       err := dispatcher.Arv.Get("containers", uuid, nil, &container)
+       if err != nil {
+               dispatcher.Logger.Warnf("error getting final container state: %v", err)
+       }
+       if container.State == dispatch.Locked || container.State == dispatch.Running {
+               dispatcher.Logger.Warnf("after %q process termination, container state for %v is %q; updating it to %q",
+                       *crunchRunCommand, uuid, container.State, dispatch.Cancelled)
+               dispatcher.UpdateState(uuid, dispatch.Cancelled)
+       }
+
+       // drain any subsequent status changes
+       for range status {
        }
 
-       log.Printf("Finished container run for %v", uuid)
+       dispatcher.Logger.Printf("finalized container %v", uuid)
+       return nil
 }