import (
"bytes"
+ "context"
"crypto/md5"
"encoding/json"
"errors"
"fmt"
"io"
+ "io/fs"
"io/ioutil"
- "log"
"math/rand"
"net/http"
"net/http/httptest"
+ "net/http/httputil"
+ "net/url"
"os"
"os/exec"
+ "path"
"regexp"
"runtime/pprof"
"strings"
"git.arvados.org/arvados.git/sdk/go/arvados"
"git.arvados.org/arvados.git/sdk/go/arvadosclient"
"git.arvados.org/arvados.git/sdk/go/arvadostest"
- "git.arvados.org/arvados.git/sdk/go/manifest"
- "golang.org/x/net/context"
. "gopkg.in/check.v1"
)
TestingT(t)
}
+const logLineStart = `(?m)(.*\n)*\d{4}-\d\d-\d\dT\d\d:\d\d:\d\d\.\d+Z `
+
var _ = Suite(&TestSuite{})
type TestSuite struct {
keepmountTmp []string
testDispatcherKeepClient KeepTestClient
testContainerKeepClient KeepTestClient
+ debian12MemoryCurrent int64
+ debian12SwapCurrent int64
+}
+
+func (s *TestSuite) SetUpSuite(c *C) {
+ buf, err := os.ReadFile("../crunchstat/testdata/debian12/sys/fs/cgroup/user.slice/user-1000.slice/session-4.scope/memory.current")
+ c.Assert(err, IsNil)
+ _, err = fmt.Sscanf(string(buf), "%d", &s.debian12MemoryCurrent)
+ c.Assert(err, IsNil)
+
+ buf, err = os.ReadFile("../crunchstat/testdata/debian12/sys/fs/cgroup/user.slice/user-1000.slice/session-4.scope/memory.swap.current")
+ c.Assert(err, IsNil)
+ _, err = fmt.Sscanf(string(buf), "%d", &s.debian12SwapCurrent)
+ c.Assert(err, IsNil)
}
func (s *TestSuite) SetUpTest(c *C) {
err = ioutil.WriteFile(s.keepmount+"/by_id/"+fakeInputCollectionPDH+"/input.json", []byte(`{"input":true}`), 0644)
c.Assert(err, IsNil)
s.runner.ArvMountPoint = s.keepmount
+ os.Setenv("InstanceType", `{"ProviderType":"a1.2xlarge","Price":1.2}`)
}
type ArvTestClient struct {
Content []arvadosclient.Dict
arvados.Container
secretMounts []byte
- Logs map[string]*bytes.Buffer
sync.Mutex
WasSetRunning bool
callraw bool
type KeepTestClient struct {
Called bool
- Content []byte
StorageClasses []string
+ blocks sync.Map
}
type stubExecutor struct {
go func() { e.exit <- e.runFunc() }()
return e.startErr
}
-func (e *stubExecutor) CgroupID() string { return "cgroupid" }
-func (e *stubExecutor) Stop() error { e.stopped = true; go func() { e.exit <- -1 }(); return e.stopErr }
-func (e *stubExecutor) Close() { e.closed = true }
+func (e *stubExecutor) Pid() int { return 1115883 } // matches pid in ../crunchstat/testdata/debian12/proc/
+func (e *stubExecutor) Stop() error { e.stopped = true; go func() { e.exit <- -1 }(); return e.stopErr }
+func (e *stubExecutor) Close() { e.closed = true }
func (e *stubExecutor) Wait(context.Context) (int, error) {
return <-e.exit, e.waitErr
}
client.Content = append(client.Content, parameters)
if resourceType == "logs" {
- et := parameters["log"].(arvadosclient.Dict)["event_type"].(string)
- if client.Logs == nil {
- client.Logs = make(map[string]*bytes.Buffer)
- }
- if client.Logs[et] == nil {
- client.Logs[et] = &bytes.Buffer{}
- }
- client.Logs[et].Write([]byte(parameters["log"].(arvadosclient.Dict)["properties"].(map[string]string)["text"]))
+ panic("logs.create called")
}
if resourceType == "collections" && output != nil {
} else if resourceType == "collections" && output != nil {
mt := parameters["collection"].(arvadosclient.Dict)["manifest_text"].(string)
output.(*arvados.Collection).UUID = uuid
- output.(*arvados.Collection).PortableDataHash = fmt.Sprintf("%x", md5.Sum([]byte(mt)))
+ output.(*arvados.Collection).PortableDataHash = arvados.PortableDataHash(mt)
}
return nil
}
var discoveryMap = map[string]interface{}{
- "defaultTrashLifetime": float64(1209600),
- "crunchLimitLogBytesPerJob": float64(67108864),
- "crunchLogThrottleBytes": float64(65536),
- "crunchLogThrottlePeriod": float64(60),
- "crunchLogThrottleLines": float64(1024),
- "crunchLogPartialLineThrottlePeriod": float64(5),
- "crunchLogBytesPerEvent": float64(4096),
- "crunchLogSecondsBetweenEvents": float64(1),
+ "crunchLogUpdateSize": float64(crunchLogUpdateSize),
+ "crunchLogUpdatePeriod": float64(crunchLogUpdatePeriod.Seconds()),
}
func (client *ArvTestClient) Discovery(key string) (interface{}, error) {
}
func (client *KeepTestClient) BlockWrite(_ context.Context, opts arvados.BlockWriteOptions) (arvados.BlockWriteResponse, error) {
- client.Content = opts.Data
+ locator := fmt.Sprintf("%x+%d", md5.Sum(opts.Data), len(opts.Data))
+ client.blocks.Store(locator, append([]byte(nil), opts.Data...))
return arvados.BlockWriteResponse{
- Locator: fmt.Sprintf("%x+%d", md5.Sum(opts.Data), len(opts.Data)),
+ Locator: locator,
}, nil
}
-func (client *KeepTestClient) ReadAt(string, []byte, int) (int, error) {
- return 0, errors.New("not implemented")
-}
-
-func (client *KeepTestClient) ClearBlockCache() {
+func (client *KeepTestClient) ReadAt(locator string, dst []byte, offset int) (int, error) {
+ loaded, ok := client.blocks.Load(locator)
+ if !ok {
+ return 0, os.ErrNotExist
+ }
+ data := loaded.([]byte)
+ if offset >= len(data) {
+ return 0, io.EOF
+ }
+ return copy(dst, data[offset:]), nil
}
func (client *KeepTestClient) Close() {
- client.Content = nil
+ client.blocks.Range(func(locator, value interface{}) bool {
+ client.blocks.Delete(locator)
+ return true
+ })
}
func (client *KeepTestClient) SetStorageClasses(sc []string) {
return errors.New("not implemented")
}
-func (client *KeepTestClient) ManifestFileReader(m manifest.Manifest, filename string) (arvados.File, error) {
- if filename == hwImageID+".tar" {
- rdr := ioutil.NopCloser(&bytes.Buffer{})
- client.Called = true
- return FileWrapper{rdr, 1321984}, nil
- } else if filename == "/file1_in_main.txt" {
- rdr := ioutil.NopCloser(strings.NewReader("foo"))
- client.Called = true
- return FileWrapper{rdr, 3}, nil
+type apiStubServer struct {
+ server *httptest.Server
+ proxy *httputil.ReverseProxy
+ intercept func(http.ResponseWriter, *http.Request) bool
+
+ container arvados.Container
+}
+
+func apiStub() (*arvados.Client, *apiStubServer) {
+ client := arvados.NewClientFromEnv()
+ apistub := &apiStubServer{}
+ apistub.server = httptest.NewTLSServer(apistub)
+ apistub.proxy = httputil.NewSingleHostReverseProxy(&url.URL{Scheme: "https", Host: client.APIHost})
+ if client.Insecure {
+ apistub.proxy.Transport = arvados.InsecureHTTPClient.Transport
}
- return nil, nil
+ client.APIHost = apistub.server.Listener.Addr().String()
+ return client, apistub
+}
+
+func (apistub *apiStubServer) ServeHTTP(w http.ResponseWriter, r *http.Request) {
+ if apistub.intercept != nil && apistub.intercept(w, r) {
+ return
+ }
+ if r.Method == "GET" && r.URL.Path == "/arvados/v1/collections/"+hwPDH {
+ json.NewEncoder(w).Encode(arvados.Collection{ManifestText: hwManifest})
+ return
+ }
+ if r.Method == "GET" && r.URL.Path == "/arvados/v1/collections/"+otherPDH {
+ json.NewEncoder(w).Encode(arvados.Collection{ManifestText: otherManifest})
+ return
+ }
+ if r.Method == "GET" && r.URL.Path == "/arvados/v1/collections/"+normalizedWithSubdirsPDH {
+ json.NewEncoder(w).Encode(arvados.Collection{ManifestText: normalizedManifestWithSubdirs})
+ return
+ }
+ if r.Method == "GET" && r.URL.Path == "/arvados/v1/collections/"+denormalizedWithSubdirsPDH {
+ json.NewEncoder(w).Encode(arvados.Collection{ManifestText: denormalizedManifestWithSubdirs})
+ return
+ }
+ if r.Method == "GET" && r.URL.Path == "/arvados/v1/containers/"+apistub.container.UUID {
+ json.NewEncoder(w).Encode(apistub.container)
+ return
+ }
+ apistub.proxy.ServeHTTP(w, r)
}
func (s *TestSuite) TestLoadImage(c *C) {
KeepTestClient
}
-func (*KeepErrorTestClient) ManifestFileReader(manifest.Manifest, string) (arvados.File, error) {
- return nil, errors.New("KeepError")
-}
-
func (*KeepErrorTestClient) BlockWrite(context.Context, arvados.BlockWriteOptions) (arvados.BlockWriteResponse, error) {
return arvados.BlockWriteResponse{}, errors.New("KeepError")
}
return 0, errors.New("KeepError")
}
-type ErrorReader struct {
- FileWrapper
-}
-
-func (ErrorReader) Read(p []byte) (n int, err error) {
- return 0, errors.New("ErrorReader")
-}
-
-func (ErrorReader) Seek(int64, int) (int64, error) {
- return 0, errors.New("ErrorReader")
-}
-
-func (KeepReadErrorTestClient) ManifestFileReader(m manifest.Manifest, filename string) (arvados.File, error) {
- return ErrorReader{}, nil
-}
-
-type ClosableBuffer struct {
- bytes.Buffer
-}
-
-func (*ClosableBuffer) Close() error {
- return nil
-}
-
-type TestLogs struct {
- Stdout ClosableBuffer
- Stderr ClosableBuffer
-}
-
-func (tl *TestLogs) NewTestLoggingWriter(logstr string) (io.WriteCloser, error) {
- if logstr == "stdout" {
- return &tl.Stdout, nil
- }
- if logstr == "stderr" {
- return &tl.Stderr, nil
- }
- return nil, errors.New("???")
-}
-
func dockerLog(fd byte, msg string) []byte {
by := []byte(msg)
header := make([]byte, 8+len(by))
return 0
}
- var logs TestLogs
- s.runner.NewLogWriter = logs.NewTestLoggingWriter
s.runner.Container.ContainerImage = arvadostest.DockerImage112PDH
s.runner.Container.Command = []string{"./hw"}
s.runner.Container.OutputStorageClasses = []string{"default"}
err = s.runner.WaitFinish()
c.Assert(err, IsNil)
- c.Check(logs.Stdout.String(), Matches, ".*Hello world\n")
- c.Check(logs.Stderr.String(), Equals, "")
+ c.Check(logFileContent(c, s.runner, "stdout.txt"), Matches, `2\S+Z Hello world\n`)
+ c.Check(logFileContent(c, s.runner, "stderr.txt"), Matches, ``)
}
func (s *TestSuite) TestCommitLogs(c *C) {
defer kc.Close()
cr, err := NewContainerRunner(s.client, api, kc, "zzzzz-zzzzz-zzzzzzzzzzzzzzz")
c.Assert(err, IsNil)
- cr.CrunchLog.Timestamper = (&TestTimestamper{}).Timestamp
+ f, err := cr.openLogFile("crunch-run")
+ c.Assert(err, IsNil)
+ cr.CrunchLog = newLogWriter(newTestTimestamper(f))
cr.CrunchLog.Print("Hello world!")
cr.CrunchLog.Print("Goodbye")
err = cr.CommitLogs()
c.Check(err, IsNil)
- c.Check(api.Calls, Equals, 2)
- c.Check(api.Content[1]["ensure_unique_name"], Equals, true)
- c.Check(api.Content[1]["collection"].(arvadosclient.Dict)["name"], Equals, "logs for zzzzz-zzzzz-zzzzzzzzzzzzzzz")
- c.Check(api.Content[1]["collection"].(arvadosclient.Dict)["manifest_text"], Equals, ". 744b2e4553123b02fa7b452ec5c18993+123 0:123:crunch-run.txt\n")
+ c.Check(api.Calls, Equals, 1)
+ c.Check(api.Content[0]["ensure_unique_name"], Equals, true)
+ c.Check(api.Content[0]["collection"].(arvadosclient.Dict)["name"], Equals, "logs for zzzzz-zzzzz-zzzzzzzzzzzzzzz")
+ c.Check(api.Content[0]["collection"].(arvadosclient.Dict)["manifest_text"], Equals, ". 744b2e4553123b02fa7b452ec5c18993+123 0:123:crunch-run.txt\n")
c.Check(*cr.LogsPDH, Equals, "63da7bdacf08c40f604daad80c261e9a+60")
}
}
return d, err
}
+ client, _ := apiStub()
s.runner.MkArvClient = func(token string) (IArvadosClient, IKeepClient, *arvados.Client, error) {
- return &ArvTestClient{secretMounts: secretMounts}, &s.testContainerKeepClient, nil, nil
+ return &ArvTestClient{secretMounts: secretMounts}, &s.testContainerKeepClient, client, nil
}
if extraMounts != nil && len(extraMounts) > 0 {
}
if err != nil {
- for k, v := range s.api.Logs {
- c.Log(k)
- c.Log(v.String())
- }
+ dumpAllLogFiles(c, s.runner)
}
return s.api, s.runner, realTemp
c.Check(s.api.CalledWith("container.exit_code", 0), NotNil)
c.Check(s.api.CalledWith("container.state", "Complete"), NotNil)
- c.Check(s.api.Logs["stdout"].String(), Matches, ".*hello world\n")
+ c.Check(logFileContent(c, s.runner, "stdout.txt"), Matches, `2\S+Z hello world\n`)
c.Check(s.testDispatcherKeepClient.StorageClasses, DeepEquals, []string{"default"})
c.Check(s.testContainerKeepClient.StorageClasses, DeepEquals, []string{"default"})
}
spotInterruptionCheckInterval = time.Second / 8
ec2MetadataBaseURL = stub.URL
+ checkedLogs := false
+ checkLogs := func() {
+ checkedLogs = true
+ c.Check(logFileContent(c, s.runner, "crunch-run.txt"), Matches, `(?ms).*Checking for spot interruptions every 125ms using instance metadata at http://.*`)
+ c.Check(logFileContent(c, s.runner, "crunch-run.txt"), Matches, `(?ms).*Error checking spot interruptions: 503 Service Unavailable.*`)
+ if failureRate == 1 {
+ c.Check(logFileContent(c, s.runner, "crunch-run.txt"), Matches, `(?ms).*Giving up on checking spot interruptions after too many consecutive failures.*`)
+ return
+ }
+ text := `Cloud provider scheduled instance stop at ` + stoptime.Load().(time.Time).Format(time.RFC3339)
+ c.Check(logFileContent(c, s.runner, "crunch-run.txt"), Matches, `(?ms).*`+text+`.*`)
+ c.Check(s.api.CalledWith("container.runtime_status.warning", "preemption notice"), NotNil)
+ c.Check(s.api.CalledWith("container.runtime_status.warningDetail", text), NotNil)
+ c.Check(s.api.CalledWith("container.runtime_status.preemptionNotice", text), NotNil)
+
+ // Check that the log collection was saved, and the
+ // container record updated with the new PDH,
+ // immediately after the preemption notice was
+ // received -- i.e., while the container is still
+ // running.
+ lastpdh := ""
+ saved := make(map[string]string) // pdh => manifest_text
+ for _, call := range s.api.Content {
+ if ctr, ok := call["container"].(arvadosclient.Dict); ok {
+ if pdh, ok := ctr["log"].(string); ok {
+ lastpdh = pdh
+ }
+ }
+ if coll, ok := call["collection"].(arvadosclient.Dict); ok {
+ mt, _ := coll["manifest_text"].(string)
+ if strings.Contains(mt, ":crunch-run.txt") {
+ saved[arvados.PortableDataHash(mt)] = mt
+ }
+ }
+ }
+ logfs, err := (&arvados.Collection{ManifestText: saved[lastpdh]}).FileSystem(s.runner.dispatcherClient, s.runner.DispatcherKeepClient)
+ c.Assert(err, IsNil)
+ log, err := fs.ReadFile(arvados.FS(logfs), "crunch-run.txt")
+ c.Check(err, IsNil)
+ c.Check(string(log), Matches, `(?ms).*\Q`+text+`\E.*`)
+ }
+
go s.runner.checkSpotInterruptionNotices()
s.fullRunHelper(c, `{
"command": ["sleep", "3"],
stoptime.Store(time.Now().Add(time.Minute).UTC())
token = "different-fake-ec2-metadata-token"
time.Sleep(time.Second)
+ checkLogs()
return 0
})
- c.Check(s.api.Logs["crunch-run"].String(), Matches, `(?ms).*Checking for spot interruptions every 125ms using instance metadata at http://.*`)
- c.Check(s.api.Logs["crunch-run"].String(), Matches, `(?ms).*Error checking spot interruptions: 503 Service Unavailable.*`)
- if failureRate == 1 {
- c.Check(s.api.Logs["crunch-run"].String(), Matches, `(?ms).*Giving up on checking spot interruptions after too many consecutive failures.*`)
- } else {
- text := `Cloud provider scheduled instance stop at ` + stoptime.Load().(time.Time).Format(time.RFC3339)
- c.Check(s.api.Logs["crunch-run"].String(), Matches, `(?ms).*`+text+`.*`)
- c.Check(s.api.CalledWith("container.runtime_status.warning", "preemption notice"), NotNil)
- c.Check(s.api.CalledWith("container.runtime_status.warningDetail", text), NotNil)
- c.Check(s.api.CalledWith("container.runtime_status.preemptionNotice", text), NotNil)
- }
+ c.Check(checkedLogs, Equals, true)
}
func (s *TestSuite) TestRunTimeExceeded(c *C) {
})
c.Check(s.api.CalledWith("container.state", "Cancelled"), NotNil)
- c.Check(s.api.Logs["crunch-run"].String(), Matches, "(?ms).*maximum run time exceeded.*")
+ c.Check(logFileContent(c, s.runner, "crunch-run.txt"), Matches, "(?ms).*maximum run time exceeded.*")
}
func (s *TestSuite) TestContainerWaitFails(c *C) {
})
c.Check(s.api.CalledWith("container.state", "Cancelled"), NotNil)
- c.Check(s.api.Logs["crunch-run"].String(), Matches, "(?ms).*Container is not running.*")
+ c.Check(logFileContent(c, s.runner, "crunch-run.txt"), Matches, "(?ms).*Container is not running.*")
}
func (s *TestSuite) TestCrunchstat(c *C) {
+ s.runner.crunchstatFakeFS = os.DirFS("../crunchstat/testdata/debian12")
s.fullRunHelper(c, `{
"command": ["sleep", "1"],
"container_image": "`+arvadostest.DockerImage112PDH+`",
c.Check(s.api.CalledWith("container.exit_code", 0), NotNil)
c.Check(s.api.CalledWith("container.state", "Complete"), NotNil)
- // We didn't actually start a container, so crunchstat didn't
- // find accounting files and therefore didn't log any stats.
- // It should have logged a "can't find accounting files"
- // message after one poll interval, though, so we can confirm
- // it's alive:
- c.Assert(s.api.Logs["crunchstat"], NotNil)
- c.Check(s.api.Logs["crunchstat"].String(), Matches, `(?ms).*cgroup stats files have not appeared after 100ms.*`)
+ c.Check(logFileContent(c, s.runner, "crunchstat.txt"), Matches, `(?ms).*`+reTimestamp+` mem \d+ swap \d+ pgmajfault \d+ rss.*`)
+ c.Check(logFileContent(c, s.runner, "hoststat.txt"), Matches, `(?ms).*`+reTimestamp+` mem \d+ swap \d+ pgmajfault \d+ rss.*`)
- // The "files never appeared" log assures us that we called
- // (*crunchstat.Reporter)Stop(), and that we set it up with
- // the correct container ID "abcde":
- c.Check(s.api.Logs["crunchstat"].String(), Matches, `(?ms).*cgroup stats files never appeared for cgroupid\n`)
+ // Check that we called (*crunchstat.Reporter)Stop().
+ c.Check(logFileContent(c, s.runner, "crunch-run.txt"), Matches, `(?ms).*`+reTimestamp+` Maximum crunch-run memory rss usage was \d+ bytes\n.*`)
}
func (s *TestSuite) TestNodeInfoLog(c *C) {
- os.Setenv("SLURMD_NODENAME", "compute2")
s.fullRunHelper(c, `{
- "command": ["sleep", "1"],
+ "command": ["true"],
"container_image": "`+arvadostest.DockerImage112PDH+`",
"cwd": ".",
"environment": {},
"runtime_constraints": {},
"state": "Locked"
}`, nil, func() int {
- time.Sleep(time.Second)
return 0
})
c.Check(s.api.CalledWith("container.exit_code", 0), NotNil)
c.Check(s.api.CalledWith("container.state", "Complete"), NotNil)
- c.Assert(s.api.Logs["node"], NotNil)
- json := s.api.Logs["node"].String()
- c.Check(json, Matches, `(?ms).*"uuid": *"zzzzz-7ekkf-2z3mc76g2q73aio".*`)
- c.Check(json, Matches, `(?ms).*"total_cpu_cores": *16.*`)
- c.Check(json, Not(Matches), `(?ms).*"info":.*`)
+ json := logFileContent(c, s.runner, "node.json")
+ c.Check(json, Matches, `(?ms).*"ProviderType": *"a1\.2xlarge".*`)
+ c.Check(json, Matches, `(?ms).*"Price": *1\.2.*`)
- c.Assert(s.api.Logs["node-info"], NotNil)
- json = s.api.Logs["node-info"].String()
- c.Check(json, Matches, `(?ms).*Host Information.*`)
- c.Check(json, Matches, `(?ms).*CPU Information.*`)
- c.Check(json, Matches, `(?ms).*Memory Information.*`)
- c.Check(json, Matches, `(?ms).*Disk Space.*`)
- c.Check(json, Matches, `(?ms).*Disk INodes.*`)
+ nodeinfo := logFileContent(c, s.runner, "node-info.txt")
+ c.Check(nodeinfo, Matches, `(?ms).*Host Information.*`)
+ c.Check(nodeinfo, Matches, `(?ms).*CPU Information.*`)
+ c.Check(nodeinfo, Matches, `(?ms).*Memory Information.*`)
+ c.Check(nodeinfo, Matches, `(?ms).*Disk Space.*`)
+ c.Check(nodeinfo, Matches, `(?ms).*Disk INodes.*`)
}
func (s *TestSuite) TestLogVersionAndRuntime(c *C) {
return 0
})
- c.Assert(s.api.Logs["crunch-run"], NotNil)
- c.Check(s.api.Logs["crunch-run"].String(), Matches, `(?ms).*crunch-run \S+ \(go\S+\) start.*`)
- c.Check(s.api.Logs["crunch-run"].String(), Matches, `(?ms).*crunch-run process has uid=\d+\(.+\) gid=\d+\(.+\) groups=\d+\(.+\)(,\d+\(.+\))*\n.*`)
- c.Check(s.api.Logs["crunch-run"].String(), Matches, `(?ms).*Executing container: zzzzz-zzzzz-zzzzzzzzzzzzzzz.*`)
- c.Check(s.api.Logs["crunch-run"].String(), Matches, `(?ms).*Using container runtime: stub.*`)
+ c.Check(logFileContent(c, s.runner, "crunch-run.txt"), Matches, `(?ms).*crunch-run \S+ \(go\S+\) start.*`)
+ c.Check(logFileContent(c, s.runner, "crunch-run.txt"), Matches, `(?ms).*crunch-run process has uid=\d+\(.+\) gid=\d+\(.+\) groups=\d+\(.+\)(,\d+\(.+\))*\n.*`)
+ c.Check(logFileContent(c, s.runner, "crunch-run.txt"), Matches, `(?ms).*Executing container: zzzzz-zzzzz-zzzzzzzzzzzzzzz.*`)
+ c.Check(logFileContent(c, s.runner, "crunch-run.txt"), Matches, `(?ms).*Using container runtime: stub.*`)
+}
+
+func (s *TestSuite) testLogRSSThresholds(c *C, ram int64, expected []int, notExpected int) {
+ s.runner.crunchstatFakeFS = os.DirFS("../crunchstat/testdata/debian12")
+ s.fullRunHelper(c, `{
+ "command": ["true"],
+ "container_image": "`+arvadostest.DockerImage112PDH+`",
+ "cwd": ".",
+ "environment": {},
+ "mounts": {"/tmp": {"kind": "tmp"} },
+ "output_path": "/tmp",
+ "priority": 1,
+ "runtime_constraints": {"ram": `+fmt.Sprintf("%d", ram)+`},
+ "state": "Locked"
+ }`, nil, func() int { return 0 })
+ logs := logFileContent(c, s.runner, "crunch-run.txt")
+ c.Log("=== crunchstat logs")
+ c.Log(logs)
+ pattern := logLineStart + `Container using over %d%% of memory \(rss %d/%d bytes\)`
+ var threshold int
+ for _, threshold = range expected {
+ c.Check(logs, Matches, fmt.Sprintf(pattern, threshold, s.debian12MemoryCurrent, ram))
+ }
+ if notExpected > threshold {
+ c.Check(logs, Not(Matches), fmt.Sprintf(pattern, notExpected, s.debian12MemoryCurrent, ram))
+ }
+}
+
+func (s *TestSuite) TestLogNoRSSThresholds(c *C) {
+ s.testLogRSSThresholds(c, s.debian12MemoryCurrent*10, []int{}, 90)
+}
+
+func (s *TestSuite) TestLogSomeRSSThresholds(c *C) {
+ onePercentRSS := s.debian12MemoryCurrent / 100
+ s.testLogRSSThresholds(c, 102*onePercentRSS, []int{90, 95}, 99)
+}
+
+func (s *TestSuite) TestLogAllRSSThresholds(c *C) {
+ s.testLogRSSThresholds(c, s.debian12MemoryCurrent, []int{90, 95, 99}, 0)
+}
+
+func (s *TestSuite) TestLogMaximaAfterRun(c *C) {
+ s.runner.crunchstatFakeFS = os.DirFS("../crunchstat/testdata/debian12")
+ s.runner.parentTemp = c.MkDir()
+ s.fullRunHelper(c, `{
+ "command": ["true"],
+ "container_image": "`+arvadostest.DockerImage112PDH+`",
+ "cwd": ".",
+ "environment": {},
+ "mounts": {"/tmp": {"kind": "tmp"} },
+ "output_path": "/tmp",
+ "priority": 1,
+ "runtime_constraints": {"ram": `+fmt.Sprintf("%d", s.debian12MemoryCurrent*10)+`},
+ "state": "Locked"
+ }`, nil, func() int { return 0 })
+ logs := logFileContent(c, s.runner, "crunch-run.txt")
+ for _, expected := range []string{
+ `Maximum disk usage was \d+%, \d+/\d+ bytes`,
+ fmt.Sprintf(`Maximum container memory swap usage was %d bytes`, s.debian12SwapCurrent),
+ `Maximum container memory pgmajfault usage was \d+ faults`,
+ fmt.Sprintf(`Maximum container memory rss usage was 10%%, %d/%d bytes`, s.debian12MemoryCurrent, s.debian12MemoryCurrent*10),
+ `Maximum crunch-run memory rss usage was \d+ bytes`,
+ } {
+ c.Check(logs, Matches, logLineStart+expected)
+ }
}
func (s *TestSuite) TestCommitNodeInfoBeforeStart(c *C) {
c.Check(s.api.CalledWith("container.exit_code", 0), NotNil)
c.Check(s.api.CalledWith("container.state", "Complete"), NotNil)
- c.Assert(s.api.Logs["container"], NotNil)
- c.Check(s.api.Logs["container"].String(), Matches, `(?ms).*container_image.*`)
+ c.Check(logFileContent(c, s.runner, "container.json"), Matches, `(?ms).*container_image.*`)
}
func (s *TestSuite) TestFullRunStderr(c *C) {
c.Check(final["container"].(arvadosclient.Dict)["exit_code"], Equals, 1)
c.Check(final["container"].(arvadosclient.Dict)["log"], NotNil)
- c.Check(s.api.Logs["stdout"].String(), Matches, ".*hello\n")
- c.Check(s.api.Logs["stderr"].String(), Matches, ".*world\n")
+ c.Check(logFileContent(c, s.runner, "stdout.txt"), Matches, ".*hello\n")
+ c.Check(logFileContent(c, s.runner, "stderr.txt"), Matches, ".*world\n")
}
func (s *TestSuite) TestFullRunDefaultCwd(c *C) {
c.Check(s.api.CalledWith("container.exit_code", 0), NotNil)
c.Check(s.api.CalledWith("container.state", "Complete"), NotNil)
- c.Log(s.api.Logs["stdout"])
- c.Check(s.api.Logs["stdout"].String(), Matches, `.*workdir=""\n`)
+ c.Check(logFileContent(c, s.runner, "stdout.txt"), Matches, `.*workdir=""`)
}
func (s *TestSuite) TestFullRunSetCwd(c *C) {
c.Check(s.api.CalledWith("container.exit_code", 0), NotNil)
c.Check(s.api.CalledWith("container.state", "Complete"), NotNil)
- c.Check(s.api.Logs["stdout"].String(), Matches, ".*/bin\n")
+ c.Check(logFileContent(c, s.runner, "stdout.txt"), Matches, ".*/bin\n")
}
func (s *TestSuite) TestFullRunSetOutputStorageClasses(c *C) {
c.Check(s.api.CalledWith("container.exit_code", 0), NotNil)
c.Check(s.api.CalledWith("container.state", "Complete"), NotNil)
- c.Check(s.api.Logs["stdout"].String(), Matches, ".*/bin\n")
+ c.Check(logFileContent(c, s.runner, "stdout.txt"), Matches, ".*/bin\n")
c.Check(s.testDispatcherKeepClient.StorageClasses, DeepEquals, []string{"foo", "bar"})
c.Check(s.testContainerKeepClient.StorageClasses, DeepEquals, []string{"foo", "bar"})
}
case err = <-done:
c.Check(err, IsNil)
}
- for k, v := range s.api.Logs {
- c.Log(k)
- c.Log(v.String(), "\n")
- }
+ dumpAllLogFiles(c, s.runner)
c.Check(s.api.CalledWith("container.log", nil), NotNil)
c.Check(s.api.CalledWith("container.state", "Cancelled"), NotNil)
- c.Check(s.api.Logs["stdout"].String(), Matches, "(?ms).*foo\n$")
+ c.Check(logFileContent(c, s.runner, "stdout.txt"), Matches, "(?ms).*foo\n$")
}
func (s *TestSuite) TestFullRunSetEnv(c *C) {
c.Check(s.api.CalledWith("container.exit_code", 0), NotNil)
c.Check(s.api.CalledWith("container.state", "Complete"), NotNil)
- c.Check(s.api.Logs["stdout"].String(), Matches, `.*map\[FROBIZ:bilbo\]\n`)
+ c.Check(logFileContent(c, s.runner, "stdout.txt"), Matches, `.*map\[FROBIZ:bilbo\]`)
}
type ArvMountCmdLine struct {
return nil, nil
}
-func stubCert(temp string) string {
+func stubCert(c *C, temp string) string {
path := temp + "/ca-certificates.crt"
- crt, _ := os.Create(path)
- crt.Close()
- arvadosclient.CertFiles = []string{path}
+ err := os.WriteFile(path, []byte{}, 0666)
+ c.Assert(err, IsNil)
+ os.Setenv("SSL_CERT_FILE", path)
return path
}
cr := s.runner
am := &ArvMountCmdLine{}
cr.RunArvMount = am.ArvMountTest
+ cr.containerClient, _ = apiStub()
cr.ContainerArvClient = &ArvTestClient{}
cr.ContainerKeepClient = &KeepTestClient{}
cr.Container.OutputStorageClasses = []string{"default"}
realTemp := c.MkDir()
certTemp := c.MkDir()
- stubCertPath := stubCert(certTemp)
+ stubCertPath := stubCert(c, certTemp)
cr.parentTemp = realTemp
i := 0
cr.CleanupDirs()
checkEmpty()
}
-
- // git_tree mounts
- {
- i = 0
- cr.ArvMountPoint = ""
- (*GitMountSuite)(nil).useTestGitServer(c)
- cr.token = arvadostest.ActiveToken
- cr.Container.Mounts = make(map[string]arvados.Mount)
- cr.Container.Mounts = map[string]arvados.Mount{
- "/tip": {
- Kind: "git_tree",
- UUID: arvadostest.Repository2UUID,
- Commit: "fd3531f42995344f36c30b79f55f27b502f3d344",
- Path: "/",
- },
- "/non-tip": {
- Kind: "git_tree",
- UUID: arvadostest.Repository2UUID,
- Commit: "5ebfab0522851df01fec11ec55a6d0f4877b542e",
- Path: "/",
- },
- }
- cr.Container.OutputPath = "/tmp"
-
- bindmounts, err := cr.SetupMounts()
- c.Check(err, IsNil)
-
- for path, mount := range bindmounts {
- c.Check(mount.ReadOnly, Equals, !cr.Container.Mounts[path].Writable, Commentf("%s %#v", path, mount))
- }
-
- data, err := ioutil.ReadFile(bindmounts["/tip"].HostPath + "/dir1/dir2/file with mode 0644")
- c.Check(err, IsNil)
- c.Check(string(data), Equals, "\000\001\002\003")
- _, err = ioutil.ReadFile(bindmounts["/tip"].HostPath + "/file only on testbranch")
- c.Check(err, FitsTypeOf, &os.PathError{})
- c.Check(os.IsNotExist(err), Equals, true)
-
- data, err = ioutil.ReadFile(bindmounts["/non-tip"].HostPath + "/dir1/dir2/file with mode 0644")
- c.Check(err, IsNil)
- c.Check(string(data), Equals, "\000\001\002\003")
- data, err = ioutil.ReadFile(bindmounts["/non-tip"].HostPath + "/file only on testbranch")
- c.Check(err, IsNil)
- c.Check(string(data), Equals, "testfile\n")
-
- cr.CleanupDirs()
- checkEmpty()
- }
}
func (s *TestSuite) TestStdout(c *C) {
})
c.Check(s.api.CalledWith("container.exit_code", 3), NotNil)
c.Check(s.api.CalledWith("container.state", "Complete"), NotNil)
- c.Check(s.api.Logs["crunch-run"].String(), Matches, `(?ms).*status code 3\n.*`)
+ c.Check(logFileContent(c, s.runner, "crunch-run.txt"), Matches, `(?ms).*`+reTimestamp+` Container exited with status code 3\n.*`)
}
func (s *TestSuite) TestFullRunSetOutput(c *C) {
c.Check(s.api.CalledWith("container.runtime_status.warning", "arv-mount: Keep write error"), NotNil)
c.Check(s.api.CalledWith("container.runtime_status.warningDetail", "Test: Keep write error: I am a teapot"), NotNil)
c.Check(s.api.CalledWith("container.state", "Complete"), NotNil)
- c.Check(s.api.Logs["crunch-run"].String(), Matches, `(?ms).*Container exited with status code 137 \(signal 9, SIGKILL\).*`)
+ c.Check(logFileContent(c, s.runner, "crunch-run.txt"), Matches, `(?ms).*`+reTimestamp+` Container exited with status code 137 \(signal 9, SIGKILL\).*`)
+ c.Check(logFileContent(c, s.runner, "arv-mount.txt"), Matches, reTimestamp+` Test: Keep write error: I am a teapot\n`)
}
func (s *TestSuite) TestStdoutWithExcludeFromOutputMountPointUnderOutputDir(c *C) {
"state": "Locked"
}`, nil, func() int { return 0 })
c.Check(s.api.CalledWith("container.state", nextState), NotNil)
- c.Check(s.api.Logs["crunch-run"].String(), Matches, "(?ms).*unable to run containers.*")
+ logs := logFileContent(c, s.runner, "crunch-run.txt")
+ c.Check(logs, Matches, "(?ms).*unable to run containers.*")
if s.runner.brokenNodeHook != "" {
- c.Check(s.api.Logs["crunch-run"].String(), Matches, "(?ms).*Running broken node hook.*")
- c.Check(s.api.Logs["crunch-run"].String(), Matches, "(?ms).*killme.*")
- c.Check(s.api.Logs["crunch-run"].String(), Not(Matches), "(?ms).*Writing /var/lock/crunch-run-broken to mark node as broken.*")
+ c.Check(logs, Matches, "(?ms).*Running broken node hook.*")
+ c.Check(logs, Matches, "(?ms).*killme.*")
+ c.Check(logs, Not(Matches), "(?ms).*Writing /var/lock/crunch-run-broken to mark node as broken.*")
} else {
- c.Check(s.api.Logs["crunch-run"].String(), Matches, "(?ms).*Writing /var/lock/crunch-run-broken to mark node as broken.*")
+ c.Check(logs, Matches, "(?ms).*Writing /var/lock/crunch-run-broken to mark node as broken.*")
}
}
}
"state": "Locked"
}`, nil, func() int { return 0 })
c.Check(s.api.CalledWith("container.state", "Cancelled"), NotNil)
- c.Check(s.api.Logs["crunch-run"].String(), Matches, "(?ms).*Possible causes:.*is missing.*")
+ c.Check(logFileContent(c, s.runner, "crunch-run.txt"), Matches, "(?ms).*Possible causes:.*is missing.*")
}
}
cr := s.runner
cr.costStartTime = now.Add(-time.Hour)
var logbuf bytes.Buffer
- cr.CrunchLog.Immediate = log.New(&logbuf, "", 0)
+ cr.CrunchLog = newLogWriter(&logbuf)
// if there's no InstanceType env var, cost is calculated as 0
os.Unsetenv("InstanceType")
// hasn't found any data), cost is calculated based on
// InstanceType env var
os.Setenv("InstanceType", `{"Price":1.2}`)
- defer os.Unsetenv("InstanceType")
cost = cr.calculateCost(now)
c.Check(cost, Equals, 1.2)
c.Check(logbuf.String(), Not(Matches), `(?ms).*changed to 2\.00 .* changed to 2\.00 .*`)
}
+func (s *TestSuite) TestSIGUSR2CostUpdate(c *C) {
+ pid := os.Getpid()
+ now := time.Now()
+ pricesJSON, err := json.Marshal([]cloud.InstancePrice{
+ {StartTime: now.Add(-4 * time.Hour), Price: 2.4},
+ {StartTime: now.Add(-2 * time.Hour), Price: 2.6},
+ })
+ c.Assert(err, IsNil)
+
+ os.Setenv("InstanceType", `{"Price":2.2}`)
+ defer func(s string) { lockdir = s }(lockdir)
+ lockdir = c.MkDir()
+
+ // We can't use s.api.CalledWith because timing differences will yield
+ // different cost values across runs. getCostUpdate iterates over API
+ // calls until it finds one that sets the cost, then writes that value
+ // to the next index of costUpdates.
+ deadline := now.Add(time.Second)
+ costUpdates := make([]float64, 2)
+ costIndex := 0
+ apiIndex := 0
+ getCostUpdate := func() {
+ for ; time.Now().Before(deadline); time.Sleep(time.Second / 10) {
+ for apiIndex < len(s.api.Content) {
+ update := s.api.Content[apiIndex]
+ apiIndex++
+ var ok bool
+ var cost float64
+ if update, ok = update["container"].(arvadosclient.Dict); !ok {
+ continue
+ }
+ if cost, ok = update["cost"].(float64); !ok {
+ continue
+ }
+ c.Logf("API call #%d updates cost to %v", apiIndex-1, cost)
+ costUpdates[costIndex] = cost
+ costIndex++
+ return
+ }
+ }
+ }
+
+ s.fullRunHelper(c, `{
+ "command": ["true"],
+ "container_image": "`+arvadostest.DockerImage112PDH+`",
+ "cwd": ".",
+ "environment": {},
+ "mounts": {"/tmp": {"kind": "tmp"} },
+ "output_path": "/tmp",
+ "priority": 1,
+ "runtime_constraints": {},
+ "state": "Locked",
+ "uuid": "zzzzz-dz642-20230320101530a"
+ }`, nil, func() int {
+ s.runner.costStartTime = now.Add(-3 * time.Hour)
+ err := syscall.Kill(pid, syscall.SIGUSR2)
+ c.Check(err, IsNil, Commentf("error sending first SIGUSR2 to runner"))
+ getCostUpdate()
+
+ err = os.WriteFile(path.Join(lockdir, pricesfile), pricesJSON, 0o700)
+ c.Check(err, IsNil, Commentf("error writing JSON prices file"))
+ err = syscall.Kill(pid, syscall.SIGUSR2)
+ c.Check(err, IsNil, Commentf("error sending second SIGUSR2 to runner"))
+ getCostUpdate()
+
+ return 0
+ })
+ // Comparing with format strings makes it easy to ignore minor variations
+ // in cost across runs while keeping diagnostics pretty.
+ c.Check(fmt.Sprintf("%.3f", costUpdates[0]), Equals, "6.600")
+ c.Check(fmt.Sprintf("%.3f", costUpdates[1]), Equals, "7.600")
+}
+
type FakeProcess struct {
cmdLine []string
}
func (fp FakeProcess) CmdlineSlice() ([]string, error) {
return fp.cmdLine, nil
}
+
+func logFileContent(c *C, cr *ContainerRunner, fnm string) string {
+ buf, err := fs.ReadFile(arvados.FS(cr.LogCollection), fnm)
+ c.Assert(err, IsNil)
+ return string(buf)
+}
+
+func dumpAllLogFiles(c *C, cr *ContainerRunner) {
+ d, err := cr.LogCollection.OpenFile("/", os.O_RDONLY, 0)
+ c.Assert(err, IsNil)
+ fis, err := d.Readdir(-1)
+ c.Assert(err, IsNil)
+ for _, fi := range fis {
+ c.Logf("=== %s", fi.Name())
+ c.Log(logFileContent(c, cr, fi.Name()))
+ }
+}