github.com/grailbio/bigslice@v0.0.0-20230519005545-30c4c12152ad/exec/bigmachine_test.go (about)

     1  // Copyright 2018 GRAIL, Inc. All rights reserved.
     2  // Use of this source code is governed by the Apache 2.0
     3  // license that can be found in the LICENSE file.
     4  
     5  package exec
     6  
     7  import (
     8  	"bytes"
     9  	"context"
    10  	"io"
    11  	"io/ioutil"
    12  	"reflect"
    13  	"strings"
    14  	"sync"
    15  	"testing"
    16  	"time"
    17  
    18  	"github.com/grailbio/base/errors"
    19  	"github.com/grailbio/base/retry"
    20  	"github.com/grailbio/bigmachine/testsystem"
    21  	"github.com/grailbio/bigslice"
    22  	"github.com/grailbio/bigslice/metrics"
    23  	"github.com/grailbio/bigslice/sliceio"
    24  )
    25  
    26  func TestBigmachineExecutor(t *testing.T) {
    27  	x, stop := bigmachineTestExecutor(1)
    28  	defer stop()
    29  
    30  	gate := make(chan struct{}, 1)
    31  	gate <- struct{}{} // one for the local invocation.
    32  	tasks, _, _ := compileFunc(func() bigslice.Slice {
    33  		<-gate
    34  		return bigslice.Const(1, []int{})
    35  	})
    36  	if got, want := len(tasks), 1; got != want {
    37  		t.Fatalf("got %v, want %v", got, want)
    38  	}
    39  	task := tasks[0]
    40  
    41  	go x.Run(task)
    42  	ctx := context.Background()
    43  	task.Lock()
    44  	gate <- struct{}{}
    45  	for task.state <= TaskRunning {
    46  		if err := task.Wait(ctx); err != nil {
    47  			t.Fatal(err)
    48  		}
    49  	}
    50  	if got, want := task.state, TaskOk; got != want {
    51  		t.Fatalf("got %v, want %v", got, want)
    52  	}
    53  	task.Unlock()
    54  
    55  	// If we run it again, it should first enter waiting/running state, and
    56  	// then Ok again. There should not be a new invocation (p=1).
    57  	go x.Run(task)
    58  	task.Lock()
    59  	for task.state <= TaskRunning {
    60  		if err := task.Wait(ctx); err != nil {
    61  			t.Fatal(err)
    62  		}
    63  	}
    64  	if got, want := task.state, TaskOk; got != want {
    65  		t.Fatalf("got %v, want %v", got, want)
    66  	}
    67  	task.Unlock()
    68  }
    69  
    70  func TestBigmachineExecutorExclusive(t *testing.T) {
    71  	x, stop := bigmachineTestExecutor(1)
    72  	defer stop()
    73  	var wg sync.WaitGroup
    74  	fn := bigslice.Func(func(i int) bigslice.Slice {
    75  		wg.Done()
    76  		return bigslice.Const(1, []int{})
    77  	})
    78  	fn = fn.Exclusive()
    79  
    80  	const N = 5
    81  	var maxIndex int
    82  	wg.Add(2 * N) //one for local invocation; one for remote
    83  	for i := 0; i < N; i++ {
    84  		inv := makeExecInvocation(fn.Invocation("<test>", i))
    85  		if ix := int(inv.Index); ix > maxIndex {
    86  			maxIndex = ix
    87  		}
    88  		slice := inv.Invoke()
    89  		tasks, err := compile(inv, slice, false)
    90  		if err != nil {
    91  			t.Fatal(err)
    92  		}
    93  		go x.Run(tasks[0])
    94  	}
    95  	wg.Wait()
    96  	var n int
    97  	for i := 1; i < maxIndex+1; i++ {
    98  		if x.managers[i] != nil {
    99  			n++
   100  		}
   101  	}
   102  	if got, want := n, N; got != want {
   103  		t.Errorf("got %v, want %v", got, want)
   104  	}
   105  }
   106  
   107  func TestBigmachineExecutorTaskExclusive(t *testing.T) {
   108  	ctx := context.Background()
   109  	x, stop := bigmachineTestExecutor(2)
   110  	defer stop()
   111  	var called, replied sync.WaitGroup
   112  	fn := bigslice.Func(func() bigslice.Slice {
   113  		var once sync.Once
   114  		slice := bigslice.Const(2, []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 10})
   115  		slice = bigslice.Map(slice, func(i int) int {
   116  			once.Do(func() {
   117  				called.Done()
   118  				replied.Wait()
   119  			})
   120  			return i
   121  		}, bigslice.Exclusive)
   122  		return slice
   123  	})
   124  	inv := makeExecInvocation(fn.Invocation("<test>"))
   125  	slice := inv.Invoke()
   126  	tasks, err := compile(inv, slice, false)
   127  	if err != nil {
   128  		t.Fatal(err)
   129  	}
   130  	if got, want := len(tasks), 2; got != want {
   131  		t.Fatalf("got %v, want %v", got, want)
   132  	}
   133  	for _, task := range tasks {
   134  		if !task.Pragma.Exclusive() {
   135  			t.Fatalf("task %v not bigslice.Exclusive", task)
   136  		}
   137  	}
   138  	called.Add(2)
   139  	replied.Add(1)
   140  	go x.Run(tasks[0])
   141  	go x.Run(tasks[1])
   142  	called.Wait()
   143  	if got, want := tasks[0].State(), TaskRunning; got != want {
   144  		t.Fatalf("got %v, want %v", got, want)
   145  	}
   146  	if got, want := tasks[1].State(), TaskRunning; got != want {
   147  		t.Fatalf("got %v, want %v", got, want)
   148  	}
   149  	replied.Done()
   150  	state, err := tasks[0].WaitState(ctx, TaskOk)
   151  	if err != nil || state != TaskOk {
   152  		t.Fatal(state, err)
   153  	}
   154  	state, err = tasks[1].WaitState(ctx, TaskOk)
   155  	if err != nil || state != TaskOk {
   156  		t.Fatal(state, err)
   157  	}
   158  }
   159  
   160  func TestBigmachineExecutorPanicCompile(t *testing.T) {
   161  	x, stop := bigmachineTestExecutor(1)
   162  	defer stop()
   163  
   164  	var count int
   165  	tasks, _, _ := compileFunc(func() bigslice.Slice {
   166  		count++
   167  		if count == 2 {
   168  			panic("hello")
   169  		}
   170  		return bigslice.Const(1, []int{})
   171  	})
   172  	run(t, x, tasks, TaskErr)
   173  }
   174  
   175  // TestBigmachineExecutorProcs verifies that using the Procs pragma properly
   176  // affects machine/proc allocation.
   177  func TestBigmachineExecutorProcs(t *testing.T) {
   178  	// Set up the test with:
   179  	// - a slice with 8 tasks
   180  	// - Procs(2) so that two procs are allocated for each task
   181  	// - a system with 12 procs, 4 procs per machine
   182  	system := testsystem.New()
   183  	system.Machineprocs = 4
   184  	ctx, cancel := context.WithCancel(context.Background())
   185  	x := newBigmachineExecutor(system)
   186  	shutdown := x.Start(&Session{
   187  		Context: ctx,
   188  		p:       12, // 3 machines
   189  		maxLoad: 1,
   190  	})
   191  	defer shutdown()
   192  	defer cancel()
   193  
   194  	// We use blockc to block completion of tasks, controlling execution for our
   195  	// test. All tasks block until we close blockc.
   196  	blockc := make(chan struct{})
   197  	fn := bigslice.Func(func() bigslice.Slice {
   198  		is := make([]int, 100)
   199  		for i := range is {
   200  			is[i] = i
   201  		}
   202  		slice := bigslice.ReaderFunc(8, func(shard int, x *int, xs []int) (int, error) {
   203  			<-blockc
   204  			const N = 10
   205  			var i int
   206  			for *x < N && i < len(xs) {
   207  				xs[i] = (shard * N) + *x
   208  				i++
   209  				*x++
   210  			}
   211  			if *x == N {
   212  				return i, sliceio.EOF
   213  			}
   214  			return i, nil
   215  		}, bigslice.Procs(1)) // Exercise Procs composition.
   216  		// Add an identity mapping to exercise pipelining.
   217  		slice = bigslice.Map(slice, func(i int) int {
   218  			return i
   219  		}, bigslice.Procs(2))
   220  		return slice
   221  	})
   222  	inv := makeExecInvocation(fn.Invocation("<test>"))
   223  	slice := inv.Invoke()
   224  	tasks, err := compile(inv, slice, false)
   225  	if err != nil {
   226  		t.Fatal(err)
   227  	}
   228  	// Verify that there is one task per shard.
   229  	if got, want := len(tasks), 8; got != want {
   230  		t.Fatalf("got %v, want %v", got, want)
   231  	}
   232  	// Verify that the proc need is propagated through the pipeline.
   233  	for _, task := range tasks {
   234  		if got, want := task.Pragma.Procs(), 2; got != want {
   235  			t.Fatalf("got %v, want %v", got, want)
   236  		}
   237  	}
   238  	// Run three tasks (needing 6 procs), and verify that two machines have been
   239  	// started on which to run them.
   240  	for _, task := range tasks[:3] {
   241  		go x.Run(task)
   242  		state, err := task.WaitState(ctx, TaskRunning)
   243  		if err != nil || state != TaskRunning {
   244  			t.Fatal(state, err)
   245  		}
   246  	}
   247  	if got, want := system.N(), 2; got != want {
   248  		t.Errorf("got %v, want %v", got, want)
   249  	}
   250  	// Run the rest of the tasks, and verify that the remaining machines have
   251  	// been started on which to run them.
   252  	//
   253  	// Note: this is racy, as we don't have a way of knowing that the executor
   254  	// has blocked because it cannot acquire a machine on which to run a task.
   255  	// If this is a problem, we'll need a better solution.
   256  	for _, task := range tasks[3:] {
   257  		go x.Run(task)
   258  		func() {
   259  			stateCtx, cancel := context.WithTimeout(ctx, 100*time.Millisecond)
   260  			defer cancel()
   261  			state, err := task.WaitState(stateCtx, TaskRunning)
   262  			if stateCtx.Err() != nil {
   263  				// We expect some tasks to not reach TaskRunning, as there are
   264  				// not enough procs to service them.
   265  				return
   266  			}
   267  			if err != nil || state != TaskRunning {
   268  				t.Fatal(state, err)
   269  			}
   270  		}()
   271  	}
   272  	if got, want := system.N(), 3; got != want {
   273  		t.Errorf("got %v, want %v", got, want)
   274  	}
   275  	// Only 6 of the 8 shards should be running at this point, occupying all
   276  	// procs.
   277  	var running int
   278  	for _, task := range tasks {
   279  		if task.State() == TaskRunning {
   280  			running++
   281  		}
   282  	}
   283  	if got, want := running, 6; got != want {
   284  		t.Errorf("got %v, want %v", got, want)
   285  	}
   286  	// Verify that everything runs to completion.
   287  	close(blockc)
   288  	for _, task := range tasks {
   289  		state, err := task.WaitState(ctx, TaskOk)
   290  		if err != nil || state != TaskOk {
   291  			t.Fatal(state, err)
   292  		}
   293  	}
   294  }
   295  
   296  func TestBigmachineExecutorPanicRun(t *testing.T) {
   297  	x, stop := bigmachineTestExecutor(1)
   298  	defer stop()
   299  
   300  	tasks, _, _ := compileFunc(func() bigslice.Slice {
   301  		slice := bigslice.Const(1, []int{123})
   302  		return bigslice.Map(slice, func(i int) int {
   303  			panic(i)
   304  		})
   305  	})
   306  	run(t, x, tasks, TaskErr)
   307  	if err := tasks[0].Err(); !errors.Match(fatalErr, err) {
   308  		t.Errorf("expected fatal error, got %v", err)
   309  	}
   310  }
   311  
   312  func TestBigmachineExecutorLost(t *testing.T) {
   313  	if testing.Short() {
   314  		t.Skip("lost executor test disabled for -short")
   315  	}
   316  	system := testsystem.New()
   317  	system.Machineprocs = 1
   318  	system.KeepalivePeriod = time.Second
   319  	system.KeepaliveTimeout = 2 * time.Second
   320  	system.KeepaliveRpcTimeout = time.Second
   321  
   322  	ctx, cancel := context.WithCancel(context.Background())
   323  	x := newBigmachineExecutor(system)
   324  	shutdown := x.Start(&Session{
   325  		Context: ctx,
   326  		p:       100,
   327  		maxLoad: 1,
   328  	})
   329  	defer shutdown()
   330  	defer cancel()
   331  
   332  	// Make sure to produce enough data that requires multiple calls to
   333  	// get. Currently the batch size is 1024. We mark it as exclusive
   334  	// to ensure that the task is executed on different machine from
   335  	// the subsequent reduce (after shuffle).
   336  	readerTasks, readerSlice, _ := compileFuncExclusive(func() bigslice.Slice {
   337  		return bigslice.ReaderFunc(1, func(shard int, n *int, col []int) (int, error) {
   338  			const N = 10000
   339  			if *n >= N {
   340  				return 0, sliceio.EOF
   341  			}
   342  			for i := range col {
   343  				col[i] = *n + i
   344  			}
   345  			*n += len(col)
   346  			return len(col), nil
   347  		}, bigslice.Exclusive)
   348  	})
   349  	readerTask := readerTasks[0]
   350  	// We need to use a result, not a regular slice, so that the tasks
   351  	// are reused across Func invocations.
   352  	readerResult := &Result{
   353  		Slice: readerSlice,
   354  		tasks: readerTasks,
   355  	}
   356  	go x.Run(readerTask)
   357  	system.Wait(1)
   358  	readerTask.Lock()
   359  	for readerTask.state != TaskOk {
   360  		if err := readerTask.Wait(ctx); err != nil {
   361  			t.Fatal(err)
   362  		}
   363  	}
   364  	readerTask.Unlock()
   365  
   366  	if !system.Kill(system.Index(0)) {
   367  		t.Fatal("could not kill machine")
   368  	}
   369  	mapTasks, _, _ := compileFunc(func() bigslice.Slice {
   370  		return bigslice.Map(readerResult, func(v int) int { return v })
   371  	})
   372  	mapTask := mapTasks[0]
   373  	go x.Run(mapTask)
   374  	if state, err := mapTask.WaitState(ctx, TaskOk); err != nil {
   375  		t.Fatal(err)
   376  	} else if state != TaskLost {
   377  		t.Fatal(state)
   378  	}
   379  
   380  	// Resubmit the task: Now it should recompute successfully
   381  	// (while allocating a new machine for it). We may have to submit
   382  	// it multiple times before the worker is marked down. (We're racing
   383  	// with the failure detector.)
   384  	readerTask.Lock()
   385  	readerTask.state = TaskInit
   386  	for readerTask.state != TaskOk {
   387  		readerTask.state = TaskInit
   388  		readerTask.Unlock()
   389  		go x.Run(readerTask)
   390  		readerTask.Lock()
   391  		if err := readerTask.Wait(ctx); err != nil {
   392  			t.Fatal(err)
   393  		}
   394  	}
   395  	readerTask.Unlock()
   396  
   397  	// Now do the same for the map task. We never killed the system
   398  	// it gets allocated on so no retries. This can take a few seconds as
   399  	// we wait for machine probation to expire.
   400  	mapTask.Set(TaskInit)
   401  	go x.Run(mapTask)
   402  	if state, err := mapTask.WaitState(ctx, TaskOk); err != nil {
   403  		t.Fatal(err)
   404  	} else if state != TaskOk {
   405  		t.Fatal(state)
   406  	}
   407  }
   408  
   409  type errorSlice struct {
   410  	bigslice.Slice
   411  	err error
   412  }
   413  
   414  func (r *errorSlice) Reader(shard int, deps []sliceio.Reader) sliceio.Reader {
   415  	return sliceio.ErrReader(r.err)
   416  }
   417  
   418  func TestBigmachineExecutorErrorRun(t *testing.T) {
   419  	x, stop := bigmachineTestExecutor(1)
   420  	defer stop()
   421  
   422  	tasks, _, _ := compileFunc(func() bigslice.Slice {
   423  		return &errorSlice{bigslice.Const(1, []int{123}), errors.New("some error")}
   424  	})
   425  	run(t, x, tasks, TaskLost)
   426  }
   427  
   428  // noExportedFields is a struct with no exported fields. gob will return an
   429  // error when trying to encode a value of this type. We use this to verify our
   430  // handling of gob-encoding errors.
   431  type noExportedFields struct {
   432  	f struct{}
   433  }
   434  
   435  // TestBigmachineExecutorGobError verifies that gob-encoding errors result in
   436  // task evaluation errors.
   437  func TestBigmachineExecutorGobError(t *testing.T) {
   438  	x, stop := bigmachineTestExecutor(1)
   439  	defer stop()
   440  
   441  	fn := bigslice.Func(func(_ noExportedFields) bigslice.Slice {
   442  		return bigslice.Const(1, []int{123})
   443  	})
   444  	inv := makeExecInvocation(fn.Invocation("", noExportedFields{}))
   445  	slice := inv.Invoke()
   446  	tasks, err := compile(inv, slice, false)
   447  	if err != nil {
   448  		panic(err)
   449  	}
   450  	run(t, x, tasks, TaskErr)
   451  	if errString := tasks[0].Err().Error(); !strings.Contains(errString, "noExportedFields") {
   452  		t.Errorf("want error that contains \"noExportedFields\", got %q", errString)
   453  	}
   454  }
   455  
   456  func TestBigmachineExecutorFatalErrorRun(t *testing.T) {
   457  	x, stop := bigmachineTestExecutor(1)
   458  	defer stop()
   459  
   460  	err := errors.E(errors.Fatal, "a fatal error")
   461  	tasks, _, _ := compileFunc(func() bigslice.Slice {
   462  		return &errorSlice{bigslice.Const(1, []int{123}), err}
   463  	})
   464  	run(t, x, tasks, TaskErr)
   465  	if got, want := errors.Recover(tasks[0].Err()).Severity, errors.Fatal; got != want {
   466  		t.Errorf("got %v, want %v", got, want)
   467  	}
   468  }
   469  
   470  // TestBigmachineExecutorFatalCombinerErrorRun verifies that fatal errors in
   471  // execution of tasks that have combiners are propagated and put the task in
   472  // TaskErr state.
   473  func TestBigmachineExecutorFatalCombinerErrorRun(t *testing.T) {
   474  	x, stop := bigmachineTestExecutor(1)
   475  	defer stop()
   476  
   477  	err := errors.E(errors.Fatal, "a fatal error")
   478  	tasks, _, _ := compileFunc(func() bigslice.Slice {
   479  		var slice bigslice.Slice
   480  		slice = &errorSlice{bigslice.Const(1, []int{123}, []int{456}), err}
   481  		// This Reduce causes the tasks compiled from the error slice to use a
   482  		// combiner.
   483  		slice = bigslice.Reduce(slice, func(i, j int) int { return i + j })
   484  		return slice
   485  	})
   486  	// depTask will be a task of the errorSlice, which has a combiner.
   487  	depTask := tasks[0].Deps[0].Head
   488  	run(t, x, []*Task{depTask}, TaskErr)
   489  	if got, want := errors.Recover(depTask.Err()).Severity, errors.Fatal; got != want {
   490  		t.Errorf("got %v, want %v", got, want)
   491  	}
   492  }
   493  
   494  func TestBigmachineCompiler(t *testing.T) {
   495  	x, stop := bigmachineTestExecutor(1)
   496  	defer stop()
   497  
   498  	tasks, slice, inv := compileFunc(func() bigslice.Slice {
   499  		return bigslice.Const(10, []int{})
   500  	})
   501  	firstTasks := tasks
   502  	run(t, x, tasks, TaskOk)
   503  	tasks, _, _ = compileFunc(func() bigslice.Slice {
   504  		return bigslice.Map(
   505  			&Result{Slice: slice, invIndex: inv.Index, tasks: firstTasks},
   506  			func(i int) int { return i * 2 },
   507  		)
   508  	})
   509  	run(t, x, tasks, TaskOk)
   510  }
   511  
   512  // TestReadRetries verifies that the reader used to read task data from worker
   513  // machines correctly retries on failures.
   514  func TestReadRetries(t *testing.T) {
   515  	const N = 100
   516  	bs := make([]byte, N)
   517  	for i := range bs {
   518  		bs[i] = byte(i)
   519  	}
   520  	// The real retry policy makes this test way too slow.
   521  	origPolicy := retryPolicy
   522  	retryPolicy = retry.Backoff(1*time.Nanosecond, 10*time.Second, 2)
   523  	defer func() {
   524  		retryPolicy = origPolicy
   525  	}()
   526  	ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
   527  	defer cancel()
   528  	// Given a reader that systematically produces errors, we make sure that we
   529  	// still read correctly and that our retry behavior is reasonable.
   530  	var (
   531  		errorReader = newErrorReader(bytes.NewReader(bs))
   532  		openerAt    = readSeekerOpenerAt{r: errorReader}
   533  		r           = newRetryReader(ctx, openerAt)
   534  		// p is our per-read buffer.
   535  		p     = make([]byte, 1)
   536  		total int
   537  		data  []byte
   538  	)
   539  	for {
   540  		n, err := r.Read(p)
   541  		if err == io.EOF {
   542  			break
   543  		}
   544  		if err, ok := err.(*errors.Error); ok && err.Kind == errors.Timeout {
   545  			// If we're properly resetting our retry frequency on successful
   546  			// reads, this test should finish quickly. If not, it will exceed
   547  			// the context timeout.
   548  			t.Fatalf("took too long; check retry behavior")
   549  		}
   550  		if err != nil {
   551  			t.Fatalf("unexpected error: %v", err)
   552  		}
   553  		total += n
   554  		data = append(data, p...)
   555  	}
   556  	if got, want := total, N; got != want {
   557  		t.Errorf("got %v, want %v", got, want)
   558  	}
   559  	if got, want := data, bs; !reflect.DeepEqual(got, want) {
   560  		t.Errorf("got %v, want %v", got, want)
   561  	}
   562  }
   563  
   564  func TestBigmachineMetrics(t *testing.T) {
   565  	counter := metrics.NewCounter()
   566  
   567  	x, stop := bigmachineTestExecutor(1)
   568  	defer stop()
   569  
   570  	tasks, _, _ := compileFunc(func() (slice bigslice.Slice) {
   571  		slice = bigslice.Const(1, []int{1, 2, 3})
   572  		slice = bigslice.Map(slice, func(ctx context.Context, i int) int {
   573  			scope := metrics.ContextScope(ctx)
   574  			counter.Incr(scope, int64(i))
   575  			return i
   576  		})
   577  		return
   578  	})
   579  	task := tasks[0]
   580  	go x.Run(task)
   581  	if _, err := task.WaitState(context.Background(), TaskOk); err != nil {
   582  		t.Fatal(err)
   583  	}
   584  	if got, want := counter.Value(&task.Scope), int64(6); got != want {
   585  		t.Errorf("got %v, want %v", got, want)
   586  	}
   587  }
   588  
   589  // readSeekerOpenerAt wraps an io.ReadSeeker to implement the openerAt
   590  // interface. It simply seeks to the desired open offset.
   591  type readSeekerOpenerAt struct {
   592  	r io.ReadSeeker
   593  }
   594  
   595  func (o readSeekerOpenerAt) OpenAt(ctx context.Context, offset int64) (io.ReadCloser, error) {
   596  	if _, err := o.r.Seek(offset, io.SeekStart); err != nil {
   597  		return nil, err
   598  	}
   599  	return ioutil.NopCloser(o.r), nil
   600  }
   601  
   602  // errorReader wraps a io.ReadSeeker and systematically returns errors when
   603  // reading.
   604  type errorReader struct {
   605  	r     io.ReadSeeker
   606  	nread int
   607  }
   608  
   609  func (r *errorReader) Read(data []byte) (int, error) {
   610  	r.nread++
   611  	if r.nread%3 != 0 {
   612  		return 0, errors.New("some error")
   613  	}
   614  	return r.r.Read(data)
   615  }
   616  
   617  func (r *errorReader) Seek(offset int64, whence int) (int64, error) {
   618  	return r.r.Seek(offset, whence)
   619  }
   620  
   621  func newErrorReader(r io.ReadSeeker) *errorReader {
   622  	return &errorReader{r: r}
   623  }
   624  
   625  func run(t *testing.T, x *bigmachineExecutor, tasks []*Task, expect TaskState) {
   626  	t.Helper()
   627  	for _, task := range tasks {
   628  		go x.Run(task)
   629  	}
   630  	for _, task := range tasks {
   631  		if _, err := task.WaitState(context.Background(), expect); err != nil {
   632  			t.Fatalf("error waiting for state %v: %v", expect, err)
   633  		}
   634  		task.Lock()
   635  		if got, want := task.state, expect; got != want {
   636  			t.Fatalf("task %v: got %v, want %v", task, got, want)
   637  		}
   638  		task.Unlock()
   639  	}
   640  }
   641  
   642  func bigmachineTestExecutor(p int) (exec *bigmachineExecutor, stop func()) {
   643  	x := newBigmachineExecutor(testsystem.New())
   644  	ctx, cancel := context.WithCancel(context.Background())
   645  	shutdown := x.Start(&Session{
   646  		Context: ctx,
   647  		p:       p,
   648  		maxLoad: 1,
   649  	})
   650  	return x, func() {
   651  		cancel()
   652  		shutdown()
   653  	}
   654  }
   655  
   656  func compileFunc(f func() bigslice.Slice) ([]*Task, bigslice.Slice, execInvocation) {
   657  	fn := bigslice.Func(f)
   658  	inv := makeExecInvocation(fn.Invocation(""))
   659  	slice := inv.Invoke()
   660  	tasks, err := compile(inv, slice, false)
   661  	if err != nil {
   662  		panic(err)
   663  	}
   664  	return tasks, slice, inv
   665  }
   666  
   667  func compileFuncExclusive(f func() bigslice.Slice) ([]*Task, bigslice.Slice, execInvocation) {
   668  	fn := bigslice.Func(f).Exclusive()
   669  	inv := makeExecInvocation(fn.Invocation(""))
   670  	slice := inv.Invoke()
   671  	tasks, err := compile(inv, slice, false)
   672  	if err != nil {
   673  		panic(err)
   674  	}
   675  	return tasks, slice, inv
   676  }