github.com/grailbio/bigslice@v0.0.0-20230519005545-30c4c12152ad/exec/session.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 "context" 9 "encoding/gob" 10 "fmt" 11 "io" 12 "net/http" 13 "os" 14 "runtime" 15 "sync" 16 "sync/atomic" 17 18 "github.com/grailbio/base/backgroundcontext" 19 "github.com/grailbio/base/diagnostic/dump" 20 "github.com/grailbio/base/eventlog" 21 "github.com/grailbio/base/limiter" 22 "github.com/grailbio/base/log" 23 "github.com/grailbio/base/status" 24 "github.com/grailbio/bigmachine" 25 "github.com/grailbio/bigslice" 26 "github.com/grailbio/bigslice/metrics" 27 "github.com/grailbio/bigslice/sliceio" 28 "github.com/grailbio/bigslice/typecheck" 29 ) 30 31 // DefaultMaxLoad is the default machine max load. 32 const DefaultMaxLoad = 0.95 33 34 func init() { 35 // Users may pass *Result as a bigslice.Slice. 36 gob.Register(&Result{}) 37 } 38 39 // Session represents a Bigslice compute session. A session shares a 40 // binary and executor, and is valid for the run of the binary. A 41 // session can run multiple bigslice functions, allowing for 42 // iterative computing. 43 // 44 // A session is started by the Start method. Some executors use 45 // may launch multiple copies of the binary: these additional binaries 46 // are called workers and Start in these Start does not return. 47 // 48 // All functions must be created before Start is called, and must be 49 // created in a deterministic order. This is provided by default when 50 // functions are created as part of package initialization. Registering 51 // toplevel functions this way is both safe and encouraged: 52 // 53 // var Computation = bigslice.Func(func(..) (slice Slice) { 54 // // Build up the computation, parameterized by the function. 55 // slice = ... 56 // slice = ... 57 // return slice 58 // }) 59 // 60 // // Possibly in another package: 61 // func main() { 62 // sess := exec.Start() 63 // if err := sess.Run(ctx, Computation, args...); err != nil { 64 // log.Fatal(err) 65 // } 66 // // Success! 67 // } 68 type Session struct { 69 context.Context 70 index int32 71 shutdown func() 72 p int 73 maxLoad float64 74 executor Executor 75 status *status.Status 76 eventer eventlog.Eventer 77 tracePath string 78 79 machineCombiners bool 80 81 tracer *tracer 82 83 mu sync.Mutex 84 // roots stores all task roots compiled by this session; 85 // used for debugging. 86 roots map[*Task]struct{} 87 } 88 89 func newSession() *Session { 90 return &Session{ 91 Context: backgroundcontext.Get(), 92 index: atomic.AddInt32(&nextSessionIndex, 1) - 1, 93 roots: make(map[*Task]struct{}), 94 eventer: eventlog.Nop{}, 95 } 96 } 97 98 // An Option represents a session configuration parameter value. 99 type Option func(s *Session) 100 101 // Local configures a session with the local in-binary executor. 102 var Local Option = func(s *Session) { 103 s.executor = newLocalExecutor() 104 } 105 106 // Bigmachine configures a session using the bigmachine executor 107 // configured with the provided system. If any params are provided, 108 // they are applied to each bigmachine allocated by Bigslice. 109 func Bigmachine(system bigmachine.System, params ...bigmachine.Param) Option { 110 return func(s *Session) { 111 s.executor = newBigmachineExecutor(system, params...) 112 } 113 } 114 115 // Parallelism configures the session with the provided target 116 // parallelism. 117 func Parallelism(p int) Option { 118 if p <= 0 { 119 panic("exec.Parallelism: p <= 0") 120 } 121 return func(s *Session) { 122 s.p = p 123 } 124 } 125 126 // MaxLoad configures the session with the provided max 127 // machine load. 128 func MaxLoad(maxLoad float64) Option { 129 if maxLoad <= 0 { 130 panic("exec.MaxLoad: maxLoad <= 0") 131 } 132 return func(s *Session) { 133 s.maxLoad = maxLoad 134 } 135 } 136 137 // Status configures the session with a status object to which 138 // run statuses are reported. 139 func Status(status *status.Status) Option { 140 return func(s *Session) { 141 s.status = status 142 143 name := fmt.Sprintf("bigslice-%02d-status", s.index) 144 dump.Register(name, func(ctx context.Context, w io.Writer) error { 145 return status.Marshal(w) 146 }) 147 } 148 } 149 150 // Eventer configures the session with an Eventer that will be used to log 151 // session events (for analytics). 152 func Eventer(e eventlog.Eventer) Option { 153 return func(s *Session) { 154 s.eventer = e 155 } 156 } 157 158 // TracePath configures the path to which a trace event file for the session 159 // will be written on shutdown. 160 func TracePath(path string) Option { 161 return func(s *Session) { 162 s.tracePath = path 163 } 164 } 165 166 // MachineCombiners is a session option that turns on machine-local 167 // combine buffers. If turned on, each combiner task that belongs to 168 // the same shard-set and runs on the same machine combines values 169 // into a single, machine-local combine buffer. This can be a big 170 // performance optimization for tasks that have low key cardinality, 171 // or a key-set with very hot keys. However, due to the way it is 172 // implemented, error recovery is currently not implemented for such 173 // tasks. 174 var MachineCombiners Option = func(s *Session) { 175 s.machineCombiners = true 176 } 177 178 // nextSessionIndex is the index of the next session that will be started by 179 // Start. In general, there should be only one session per process, but we 180 // violate this in some tests. 181 var nextSessionIndex int32 182 183 // TODO(jcharumilind): Make it generally safe/sensible to have multiple 184 // sessions in the same process. 185 186 // Start creates and starts a new bigslice session, configuring it 187 // according to the provided options. Only one session may be created 188 // in a single binary invocation. The returned session remains valid for 189 // the lifetime of the binary. If no executor is configured, the session 190 // is configured to use the bigmachine executor. 191 func Start(options ...Option) *Session { 192 s := newSession() 193 for _, opt := range options { 194 opt(s) 195 } 196 if s.p == 0 { 197 s.p = 1 198 } 199 if s.maxLoad == 0 { 200 s.maxLoad = DefaultMaxLoad 201 } 202 if s.executor == nil { 203 s.executor = newBigmachineExecutor(bigmachine.Local) 204 } 205 s.start() 206 return s 207 } 208 209 // Run evaluates the slice returned by the bigslice func funcv 210 // applied to the provided arguments. Tasks are run by the session's 211 // executor. Run returns when the computation has completed, or else 212 // on error. It is safe to make concurrent calls to Run; the 213 // underlying computation will be performed in parallel. 214 func (s *Session) Run(ctx context.Context, funcv *bigslice.FuncValue, args ...interface{}) (*Result, error) { 215 return s.run(ctx, 1, funcv, args...) 216 } 217 218 // Must is a version of Run that panics if the computation fails. 219 func (s *Session) Must(ctx context.Context, funcv *bigslice.FuncValue, args ...interface{}) *Result { 220 res, err := s.run(ctx, 1, funcv, args...) 221 if err != nil { 222 log.Panicf("exec.Run: %v", err) 223 } 224 return res 225 } 226 227 // Discard discards the storage resources held by the subgraph given by roots. 228 // This should be used to discard tasks whose results are no longer needed. If 229 // the task results are needed by another computation, they will be recomputed. 230 // Discarding is best-effort, so no error is returned. 231 func (s *Session) Discard(ctx context.Context, roots []*Task) { 232 var ( 233 limiter = limiter.New() 234 wg sync.WaitGroup 235 ) 236 limiter.Release(64) 237 // Best effort, so discard error. 238 _ = iterTasks(roots, func(task *Task) error { 239 if err := limiter.Acquire(ctx, 1); err != nil { 240 return err 241 } 242 wg.Add(1) 243 go func() { 244 defer wg.Done() 245 defer limiter.Release(1) 246 s.executor.Discard(ctx, task) 247 }() 248 return nil 249 }) 250 wg.Wait() 251 } 252 253 func (s *Session) start() { 254 s.shutdown = s.executor.Start(s) 255 log.Printf("eventlog: %s", s.eventer) 256 s.eventer.Event("bigslice:sessionStart", 257 "command", command(), 258 "executorType", s.executor.Name(), 259 "parallelism", s.p, 260 "maxLoad", s.maxLoad, 261 "machineCombiners", s.machineCombiners) 262 s.tracer = newTracer() 263 264 name := fmt.Sprintf("bigslice-%02d-trace", s.index) 265 dump.Register(name, func(ctx context.Context, w io.Writer) error { 266 return s.tracer.Marshal(w) 267 }) 268 } 269 270 // statusMu is used to prevent interleaving of slice and task status groups. 271 // Unrelated status groups may be interleaved, but we are at least internally 272 // consistent. 273 var statusMu sync.Mutex 274 275 func (s *Session) run(ctx context.Context, calldepth int, funcv *bigslice.FuncValue, args ...interface{}) (*Result, error) { 276 location := "<unknown>" 277 if _, file, line, ok := runtime.Caller(calldepth + 1); ok { 278 location = fmt.Sprintf("%s:%d", file, line) 279 defer typecheck.Location(file, line) 280 } 281 var ( 282 inv execInvocation 283 slice bigslice.Slice 284 tasks []*Task 285 sliceGroup *status.Group 286 taskGroup *status.Group 287 ) 288 // Make invocation and status setup atomic so that status displays in 289 // invocation index order. 290 // 291 // TODO(jcharumilind): Add functionality to status package to control 292 // ordering. 293 err := func() error { 294 statusMu.Lock() 295 defer statusMu.Unlock() 296 inv = makeExecInvocation(funcv.Invocation(location, args...)) 297 slice = inv.Invoke() 298 var err error 299 tasks, err = compile(inv, slice, s.machineCombiners) 300 if err != nil { 301 return err 302 } 303 // Freeze the environment to ensure that compilations are consistent 304 // (e.g. across workers). 305 inv.Env.Freeze() 306 // TODO(marius): give a way to provide names for these groups 307 if s.status != nil { 308 // Make the slice status group come before the more granular task 309 // status group, as we generally want increasing level of detail 310 // when observing status. 311 sliceGroup = s.status.Groupf("run %s [%d] slices", location, inv.Index) 312 _ = s.status.Groups() 313 // taskGroup is managed by Eval. 314 taskGroup = s.status.Groupf("run %s [%d] tasks", location, inv.Index) 315 _ = s.status.Groups() 316 } 317 return nil 318 }() 319 if err != nil { 320 return nil, err 321 } 322 if sliceGroup != nil { 323 maintainCtx, cancel := context.WithCancel(ctx) 324 defer cancel() 325 go maintainSliceGroup(maintainCtx, tasks, sliceGroup) 326 } 327 logInvocationCtx, logInvocationCancel := context.WithCancel(ctx) 328 defer logInvocationCancel() 329 go logInvocation(logInvocationCtx, inv, tasks) 330 // Register all the tasks so they may be used in visualization. 331 s.mu.Lock() 332 for _, task := range tasks { 333 s.roots[task] = struct{}{} 334 } 335 s.mu.Unlock() 336 err = Eval(ctx, s.executor, tasks, taskGroup) 337 errString := "success" 338 if err != nil { 339 errString = err.Error() 340 } 341 log.Printf("invocation: %s(%d) done: %s", inv.Location, inv.Index, errString) 342 return &Result{ 343 Slice: slice, 344 sess: s, 345 invIndex: inv.Index, 346 tasks: tasks, 347 }, err 348 } 349 350 // Parallelism returns the desired amount of evaluation parallelism. 351 func (s *Session) Parallelism() int { 352 return s.p 353 } 354 355 // MaxLoad returns the maximum load on each allocated machine. 356 func (s *Session) MaxLoad() float64 { 357 return s.maxLoad 358 } 359 360 // Shutdown tears down resources associated with this session. 361 // It should be called when the session is discarded. 362 func (s *Session) Shutdown() { 363 if s.shutdown != nil { 364 s.shutdown() 365 } 366 if s.tracePath != "" { 367 writeTraceFile(s.tracer, s.tracePath) 368 } 369 } 370 371 // Status returns the session's status aggregator. 372 func (s *Session) Status() *status.Status { 373 return s.status 374 } 375 376 func (s *Session) HandleDebug(handler *http.ServeMux) { 377 s.executor.HandleDebug(handler) 378 handler.Handle("/debug", http.HandlerFunc(s.handleDebug)) 379 handler.Handle("/debug/tasks/graph", http.HandlerFunc(s.handleTasksGraph)) 380 handler.Handle("/debug/tasks", http.HandlerFunc(s.handleTasks)) 381 if s.tracer != nil { 382 handler.HandleFunc("/debug/trace", func(w http.ResponseWriter, r *http.Request) { 383 w.Header().Add("content-type", "application/json; charset=utf-8") 384 if err := s.tracer.Marshal(w); err != nil { 385 log.Error.Printf("exec.Session: /debug/trace: marshal: %v", err) 386 } 387 }) 388 } 389 } 390 391 // A Result is the output of a Slice evaluation. It is the only type 392 // implementing bigslice.Slice that is a legal argument to a 393 // bigslice.Func. 394 type Result struct { 395 bigslice.Slice 396 invIndex uint64 397 sess *Session 398 tasks []*Task 399 initScope sync.Once 400 scope metrics.Scope 401 } 402 403 // Scanner returns a scanner that scans the output. If the output contains 404 // multiple shards, they are scanned sequentially. You must call Close on the 405 // returned scanner when you are done scanning. You may get and scan multiple 406 // scanners concurrently from r. 407 func (r *Result) Scanner() *sliceio.Scanner { 408 reader := r.open() 409 return sliceio.NewScanner(r, reader) 410 } 411 412 // Scope returns the merged metrics scope for the entire task graph represented 413 // by the result r. Scope relies on the local values in the scopes of the task 414 // graph, and thus are not precise. 415 // 416 // TODO(marius): flow and merge scopes along with data to provide precise 417 // metrics. 418 func (r *Result) Scope() *metrics.Scope { 419 r.initScope.Do(func() { 420 _ = iterTasks(r.tasks, func(task *Task) error { 421 r.scope.Merge(&task.Scope) 422 return nil 423 }) 424 }) 425 return &r.scope 426 } 427 428 func (r *Result) open() sliceio.ReadCloser { 429 readers := make([]sliceio.ReadCloser, len(r.tasks)) 430 for i := range readers { 431 readers[i] = r.sess.executor.Reader(r.tasks[i], 0) 432 } 433 return sliceio.MultiReader(readers...) 434 } 435 436 // Discard discards the storage resources held by the subgraph of tasks used to 437 // compute r. This should be used to discard results that are no longer needed. 438 // If the results are needed by another computation, they will be recomputed. 439 // Discarding is best-effort, so no error is returned. 440 func (r *Result) Discard(ctx context.Context) { 441 r.sess.Discard(ctx, r.tasks) 442 } 443 444 func writeTraceFile(tracer *tracer, path string) { 445 w, err := os.Create(path) 446 if err != nil { 447 log.Error.Printf("error creating trace file at %q: %v", path, err) 448 return 449 } 450 defer func() { 451 if closeErr := w.Close(); closeErr != nil { 452 log.Error.Printf("error closing trace file at %q: %v", path, closeErr) 453 return 454 } 455 }() 456 err = tracer.Marshal(w) 457 if err != nil { 458 log.Error.Printf("error marshaling to trace file at %q: %v", path, err) 459 return 460 } 461 }