// Copyright 2023 The Go Authors. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.

package trace

import (
	
	

	
	
	
)

// ordering emulates Go scheduler state for both validation and
// for putting events in the right order.
type ordering struct {
	gStates     map[GoID]*gState
	pStates     map[ProcID]*pState // TODO: The keys are dense, so this can be a slice.
	mStates     map[ThreadID]*mState
	activeTasks map[TaskID]taskState
	gcSeq       uint64
	gcState     gcState
	initialGen  uint64

	// Some events like GoDestroySyscall produce two events instead of one.
	// extraEvent is this extra space. advance must not be called unless
	// the extraEvent has been consumed with consumeExtraEvent.
	//
	// TODO(mknyszek): Replace this with a more formal queue.
	extraEvent Event
}

// consumeExtraEvent consumes the extra event.
func ( *ordering) () Event {
	if .extraEvent.Kind() == EventBad {
		return Event{}
	}
	 := .extraEvent
	.extraEvent = Event{}
	return 
}

// advance checks if it's valid to proceed with ev which came from thread m.
//
// Returns the schedCtx at the point of the event, whether it's OK to advance
// with this event, and any error encountered in validation.
//
// It assumes the gen value passed to it is monotonically increasing across calls.
//
// If any error is returned, then the trace is broken and trace parsing must cease.
// If it's not valid to advance with ev, but no error was encountered, the caller
// should attempt to advance with other candidate events from other threads. If the
// caller runs out of candidates, the trace is invalid.
func ( *ordering) ( *baseEvent,  *evTable,  ThreadID,  uint64) (schedCtx, bool, error) {
	if .initialGen == 0 {
		// Set the initial gen if necessary.
		.initialGen = 
	}

	var ,  schedCtx
	.M = 
	.M = 

	if  == NoThread {
		.P = NoProc
		.G = NoGoroutine
		 = 
	} else {
		// Pull out or create the mState for this event.
		,  := .mStates[]
		if ! {
			 = &mState{
				g: NoGoroutine,
				p: NoProc,
			}
			.mStates[] = 
		}
		.P = .p
		.G = .g
		 = 
		defer func() {
			// Update the mState for this event.
			.p = .P
			.g = .G
		}()
	}

	switch  := .typ;  {
	// Handle procs.
	case go122.EvProcStatus:
		 := ProcID(.args[0])
		 := go122.ProcStatus(.args[1])
		if int() >= len(go122ProcStatus2ProcState) {
			return , false, fmt.Errorf("invalid status for proc %d: %d", , )
		}
		 := go122ProcStatus2ProcState[]
		if ,  := .pStates[];  {
			if  == go122.ProcSyscallAbandoned && .status == go122.ProcSyscall {
				// ProcSyscallAbandoned is a special case of ProcSyscall. It indicates a
				// potential loss of information, but if we're already in ProcSyscall,
				// we haven't lost the relevant information. Promote the status and advance.
				 = ProcRunning
				.args[1] = uint64(go122.ProcSyscall)
			} else if  == go122.ProcSyscallAbandoned && .status == go122.ProcSyscallAbandoned {
				// If we're passing through ProcSyscallAbandoned, then there's no promotion
				// to do. We've lost the M that this P is associated with. However it got there,
				// it's going to appear as idle in the API, so pass through as idle.
				 = ProcIdle
				.args[1] = uint64(go122.ProcSyscallAbandoned)
			} else if .status !=  {
				return , false, fmt.Errorf("inconsistent status for proc %d: old %v vs. new %v", , .status, )
			}
			.seq = makeSeq(, 0) // Reset seq.
		} else {
			.pStates[] = &pState{id: , status: , seq: makeSeq(, 0)}
			if  == .initialGen {
				 = ProcUndetermined
			} else {
				 = ProcNotExist
			}
		}
		.extra(version.Go122)[0] = uint64() // Smuggle in the old state for StateTransition.

		// Bind the proc to the new context, if it's running.
		if  == go122.ProcRunning ||  == go122.ProcSyscall {
			.P = 
		}
		// If we're advancing through ProcSyscallAbandoned *but* oldState is running then we've
		// promoted it to ProcSyscall. However, because it's ProcSyscallAbandoned, we know this
		// P is about to get stolen and its status very likely isn't being emitted by the same
		// thread it was bound to. Since this status is Running -> Running and Running is binding,
		// we need to make sure we emit it in the right context: the context to which it is bound.
		// Find it, and set our current context to it.
		if  == go122.ProcSyscallAbandoned &&  == ProcRunning {
			// N.B. This is slow but it should be fairly rare.
			 := false
			for ,  := range .mStates {
				if .p ==  {
					.M = 
					.P = 
					.G = .g
					 = true
				}
			}
			if ! {
				return , false, fmt.Errorf("failed to find sched context for proc %d that's about to be stolen", )
			}
		}
		return , true, nil
	case go122.EvProcStart:
		 := ProcID(.args[0])
		 := makeSeq(, .args[1])

		// Try to advance. We might fail here due to sequencing, because the P hasn't
		// had a status emitted, or because we already have a P and we're in a syscall,
		// and we haven't observed that it was stolen from us yet.
		,  := .pStates[]
		if ! || .status != go122.ProcIdle || !.succeeds(.seq) || .P != NoProc {
			// We can't make an inference as to whether this is bad. We could just be seeing
			// a ProcStart on a different M before the proc's state was emitted, or before we
			// got to the right point in the trace.
			//
			// Note that we also don't advance here if we have a P and we're in a syscall.
			return , false, nil
		}
		// We can advance this P. Check some invariants.
		//
		// We might have a goroutine if a goroutine is exiting a syscall.
		 := event.SchedReqs{Thread: event.MustHave, Proc: event.MustNotHave, Goroutine: event.MayHave}
		if  := validateCtx(, );  != nil {
			return , false, 
		}
		.status = go122.ProcRunning
		.seq = 
		.P = 
		return , true, nil
	case go122.EvProcStop:
		// We must be able to advance this P.
		//
		// There are 2 ways a P can stop: ProcStop and ProcSteal. ProcStop is used when the P
		// is stopped by the same M that started it, while ProcSteal is used when another M
		// steals the P by stopping it from a distance.
		//
		// Since a P is bound to an M, and we're stopping on the same M we started, it must
		// always be possible to advance the current M's P from a ProcStop. This is also why
		// ProcStop doesn't need a sequence number.
		,  := .pStates[.P]
		if ! {
			return , false, fmt.Errorf("event %s for proc (%v) that doesn't exist", go122.EventString(), .P)
		}
		if .status != go122.ProcRunning && .status != go122.ProcSyscall {
			return , false, fmt.Errorf("%s event for proc that's not %s or %s", go122.EventString(), go122.ProcRunning, go122.ProcSyscall)
		}
		 := event.SchedReqs{Thread: event.MustHave, Proc: event.MustHave, Goroutine: event.MayHave}
		if  := validateCtx(, );  != nil {
			return , false, 
		}
		.status = go122.ProcIdle
		.P = NoProc
		return , true, nil
	case go122.EvProcSteal:
		 := ProcID(.args[0])
		 := makeSeq(, .args[1])
		,  := .pStates[]
		if ! || (.status != go122.ProcSyscall && .status != go122.ProcSyscallAbandoned) || !.succeeds(.seq) {
			// We can't make an inference as to whether this is bad. We could just be seeing
			// a ProcStart on a different M before the proc's state was emitted, or before we
			// got to the right point in the trace.
			return , false, nil
		}
		// We can advance this P. Check some invariants.
		 := event.SchedReqs{Thread: event.MustHave, Proc: event.MayHave, Goroutine: event.MayHave}
		if  := validateCtx(, );  != nil {
			return , false, 
		}
		// Smuggle in the P state that let us advance so we can surface information to the event.
		// Specifically, we need to make sure that the event is interpreted not as a transition of
		// ProcRunning -> ProcIdle but ProcIdle -> ProcIdle instead.
		//
		// ProcRunning is binding, but we may be running with a P on the current M and we can't
		// bind another P. This P is about to go ProcIdle anyway.
		 := .status
		.extra(version.Go122)[0] = uint64()

		// Update the P's status and sequence number.
		.status = go122.ProcIdle
		.seq = 

		// If we've lost information then don't try to do anything with the M.
		// It may have moved on and we can't be sure.
		if  == go122.ProcSyscallAbandoned {
			return , true, nil
		}

		// Validate that the M we're stealing from is what we expect.
		 := ThreadID(.args[2]) // The M we're stealing from.

		if  == .M {
			// We're stealing from ourselves. This behaves like a ProcStop.
			if .P !=  {
				return , false, fmt.Errorf("tried to self-steal proc %d (thread %d), but got proc %d instead", , , .P)
			}
			.P = NoProc
			return , true, nil
		}

		// We're stealing from some other M.
		,  := .mStates[]
		if ! {
			return , false, fmt.Errorf("stole proc from non-existent thread %d", )
		}

		// Make sure we're actually stealing the right P.
		if .p !=  {
			return , false, fmt.Errorf("tried to steal proc %d from thread %d, but got proc %d instead", , , .p)
		}

		// Tell the M it has no P so it can proceed.
		//
		// This is safe because we know the P was in a syscall and
		// the other M must be trying to get out of the syscall.
		// GoSyscallEndBlocked cannot advance until the corresponding
		// M loses its P.
		.p = NoProc
		return , true, nil

	// Handle goroutines.
	case go122.EvGoStatus:
		 := GoID(.args[0])
		 := ThreadID(.args[1])
		 := go122.GoStatus(.args[2])

		if int() >= len(go122GoStatus2GoState) {
			return , false, fmt.Errorf("invalid status for goroutine %d: %d", , )
		}
		 := go122GoStatus2GoState[]
		if ,  := .gStates[];  {
			if .status !=  {
				return , false, fmt.Errorf("inconsistent status for goroutine %d: old %v vs. new %v", , .status, )
			}
			.seq = makeSeq(, 0) // Reset seq.
		} else if  == .initialGen {
			// Set the state.
			.gStates[] = &gState{id: , status: , seq: makeSeq(, 0)}
			 = GoUndetermined
		} else {
			return , false, fmt.Errorf("found goroutine status for new goroutine after the first generation: id=%v status=%v", , )
		}
		.extra(version.Go122)[0] = uint64() // Smuggle in the old state for StateTransition.

		switch  {
		case go122.GoRunning:
			// Bind the goroutine to the new context, since it's running.
			.G = 
		case go122.GoSyscall:
			if  == NoThread {
				return , false, fmt.Errorf("found goroutine %d in syscall without a thread", )
			}
			// Is the syscall on this thread? If so, bind it to the context.
			// Otherwise, we're talking about a G sitting in a syscall on an M.
			// Validate the named M.
			if  == .M {
				if  != .initialGen && .G !=  {
					// If this isn't the first generation, we *must* have seen this
					// binding occur already. Even if the G was blocked in a syscall
					// for multiple generations since trace start, we would have seen
					// a previous GoStatus event that bound the goroutine to an M.
					return , false, fmt.Errorf("inconsistent thread for syscalling goroutine %d: thread has goroutine %d", , .G)
				}
				.G = 
				break
			}
			// Now we're talking about a thread and goroutine that have been
			// blocked on a syscall for the entire generation. This case must
			// not have a P; the runtime makes sure that all Ps are traced at
			// the beginning of a generation, which involves taking a P back
			// from every thread.
			,  := .mStates[]
			if  {
				// This M has been seen. That means we must have seen this
				// goroutine go into a syscall on this thread at some point.
				if .g !=  {
					// But the G on the M doesn't match. Something's wrong.
					return , false, fmt.Errorf("inconsistent thread for syscalling goroutine %d: thread has goroutine %d", , .g)
				}
				// This case is just a Syscall->Syscall event, which needs to
				// appear as having the G currently bound to this M.
				.G = .g
			} else if ! {
				// The M hasn't been seen yet. That means this goroutine
				// has just been sitting in a syscall on this M. Create
				// a state for it.
				.mStates[] = &mState{g: , p: NoProc}
				// Don't set curCtx.G in this case because this event is the
				// binding event (and curCtx represents the "before" state).
			}
			// Update the current context to the M we're talking about.
			.M = 
		}
		return , true, nil
	case go122.EvGoCreate:
		// Goroutines must be created on a running P, but may or may not be created
		// by a running goroutine.
		 := event.SchedReqs{Thread: event.MustHave, Proc: event.MustHave, Goroutine: event.MayHave}
		if  := validateCtx(, );  != nil {
			return , false, 
		}
		// If we have a goroutine, it must be running.
		if ,  := .gStates[.G];  && .status != go122.GoRunning {
			return , false, fmt.Errorf("%s event for goroutine that's not %s", go122.EventString(), GoRunning)
		}
		// This goroutine created another. Add a state for it.
		 := GoID(.args[0])
		if ,  := .gStates[];  {
			return , false, fmt.Errorf("tried to create goroutine (%v) that already exists", )
		}
		.gStates[] = &gState{id: , status: go122.GoRunnable, seq: makeSeq(, 0)}
		return , true, nil
	case go122.EvGoDestroy, go122.EvGoStop, go122.EvGoBlock:
		// These are goroutine events that all require an active running
		// goroutine on some thread. They must *always* be advance-able,
		// since running goroutines are bound to their M.
		if  := validateCtx(, event.UserGoReqs);  != nil {
			return , false, 
		}
		,  := .gStates[.G]
		if ! {
			return , false, fmt.Errorf("event %s for goroutine (%v) that doesn't exist", go122.EventString(), .G)
		}
		if .status != go122.GoRunning {
			return , false, fmt.Errorf("%s event for goroutine that's not %s", go122.EventString(), GoRunning)
		}
		// Handle each case slightly differently; we just group them together
		// because they have shared preconditions.
		switch  {
		case go122.EvGoDestroy:
			// This goroutine is exiting itself.
			delete(.gStates, .G)
			.G = NoGoroutine
		case go122.EvGoStop:
			// Goroutine stopped (yielded). It's runnable but not running on this M.
			.status = go122.GoRunnable
			.G = NoGoroutine
		case go122.EvGoBlock:
			// Goroutine blocked. It's waiting now and not running on this M.
			.status = go122.GoWaiting
			.G = NoGoroutine
		}
		return , true, nil
	case go122.EvGoStart:
		 := GoID(.args[0])
		 := makeSeq(, .args[1])
		,  := .gStates[]
		if ! || .status != go122.GoRunnable || !.succeeds(.seq) {
			// We can't make an inference as to whether this is bad. We could just be seeing
			// a GoStart on a different M before the goroutine was created, before it had its
			// state emitted, or before we got to the right point in the trace yet.
			return , false, nil
		}
		// We can advance this goroutine. Check some invariants.
		 := event.SchedReqs{Thread: event.MustHave, Proc: event.MustHave, Goroutine: event.MustNotHave}
		if  := validateCtx(, );  != nil {
			return , false, 
		}
		.status = go122.GoRunning
		.seq = 
		.G = 
		return , true, nil
	case go122.EvGoUnblock:
		// N.B. These both reference the goroutine to unblock, not the current goroutine.
		 := GoID(.args[0])
		 := makeSeq(, .args[1])
		,  := .gStates[]
		if ! || .status != go122.GoWaiting || !.succeeds(.seq) {
			// We can't make an inference as to whether this is bad. We could just be seeing
			// a GoUnblock on a different M before the goroutine was created and blocked itself,
			// before it had its state emitted, or before we got to the right point in the trace yet.
			return , false, nil
		}
		.status = go122.GoRunnable
		.seq = 
		// N.B. No context to validate. Basically anything can unblock
		// a goroutine (e.g. sysmon).
		return , true, nil
	case go122.EvGoSyscallBegin:
		// Entering a syscall requires an active running goroutine with a
		// proc on some thread. It is always advancable.
		if  := validateCtx(, event.UserGoReqs);  != nil {
			return , false, 
		}
		,  := .gStates[.G]
		if ! {
			return , false, fmt.Errorf("event %s for goroutine (%v) that doesn't exist", go122.EventString(), .G)
		}
		if .status != go122.GoRunning {
			return , false, fmt.Errorf("%s event for goroutine that's not %s", go122.EventString(), GoRunning)
		}
		// Goroutine entered a syscall. It's still running on this P and M.
		.status = go122.GoSyscall
		,  := .pStates[.P]
		if ! {
			return , false, fmt.Errorf("uninitialized proc %d found during %s", .P, go122.EventString())
		}
		.status = go122.ProcSyscall
		// Validate the P sequence number on the event and advance it.
		//
		// We have a P sequence number for what is supposed to be a goroutine event
		// so that we can correctly model P stealing. Without this sequence number here,
		// the syscall from which a ProcSteal event is stealing can be ambiguous in the
		// face of broken timestamps. See the go122-syscall-steal-proc-ambiguous test for
		// more details.
		//
		// Note that because this sequence number only exists as a tool for disambiguation,
		// we can enforce that we have the right sequence number at this point; we don't need
		// to back off and see if any other events will advance. This is a running P.
		 := makeSeq(, .args[0])
		if !.succeeds(.seq) {
			return , false, fmt.Errorf("failed to advance %s: can't make sequence: %s -> %s", go122.EventString(), .seq, )
		}
		.seq = 
		return , true, nil
	case go122.EvGoSyscallEnd:
		// This event is always advance-able because it happens on the same
		// thread that EvGoSyscallStart happened, and the goroutine can't leave
		// that thread until its done.
		if  := validateCtx(, event.UserGoReqs);  != nil {
			return , false, 
		}
		,  := .gStates[.G]
		if ! {
			return , false, fmt.Errorf("event %s for goroutine (%v) that doesn't exist", go122.EventString(), .G)
		}
		if .status != go122.GoSyscall {
			return , false, fmt.Errorf("%s event for goroutine that's not %s", go122.EventString(), GoRunning)
		}
		.status = go122.GoRunning

		// Transfer the P back to running from syscall.
		,  := .pStates[.P]
		if ! {
			return , false, fmt.Errorf("uninitialized proc %d found during %s", .P, go122.EventString())
		}
		if .status != go122.ProcSyscall {
			return , false, fmt.Errorf("expected proc %d in state %v, but got %v instead", .P, go122.ProcSyscall, .status)
		}
		.status = go122.ProcRunning
		return , true, nil
	case go122.EvGoSyscallEndBlocked:
		// This event becomes advanceable when its P is not in a syscall state
		// (lack of a P altogether is also acceptable for advancing).
		// The transfer out of ProcSyscall can happen either voluntarily via
		// ProcStop or involuntarily via ProcSteal. We may also acquire a new P
		// before we get here (after the transfer out) but that's OK: that new
		// P won't be in the ProcSyscall state anymore.
		//
		// Basically: while we have a preemptible P, don't advance, because we
		// *know* from the event that we're going to lose it at some point during
		// the syscall. We shouldn't advance until that happens.
		if .P != NoProc {
			,  := .pStates[.P]
			if ! {
				return , false, fmt.Errorf("uninitialized proc %d found during %s", .P, go122.EventString())
			}
			if .status == go122.ProcSyscall {
				return , false, nil
			}
		}
		// As mentioned above, we may have a P here if we ProcStart
		// before this event.
		if  := validateCtx(, event.SchedReqs{Thread: event.MustHave, Proc: event.MayHave, Goroutine: event.MustHave});  != nil {
			return , false, 
		}
		,  := .gStates[.G]
		if ! {
			return , false, fmt.Errorf("event %s for goroutine (%v) that doesn't exist", go122.EventString(), .G)
		}
		if .status != go122.GoSyscall {
			return , false, fmt.Errorf("%s event for goroutine that's not %s", go122.EventString(), GoRunning)
		}
		.G = NoGoroutine
		.status = go122.GoRunnable
		return , true, nil
	case go122.EvGoCreateSyscall:
		// This event indicates that a goroutine is effectively
		// being created out of a cgo callback. Such a goroutine
		// is 'created' in the syscall state.
		if  := validateCtx(, event.SchedReqs{Thread: event.MustHave, Proc: event.MayHave, Goroutine: event.MustNotHave});  != nil {
			return , false, 
		}
		// This goroutine is effectively being created. Add a state for it.
		 := GoID(.args[0])
		if ,  := .gStates[];  {
			return , false, fmt.Errorf("tried to create goroutine (%v) in syscall that already exists", )
		}
		.gStates[] = &gState{id: , status: go122.GoSyscall, seq: makeSeq(, 0)}
		// Goroutine is executing. Bind it to the context.
		.G = 
		return , true, nil
	case go122.EvGoDestroySyscall:
		// This event indicates that a goroutine created for a
		// cgo callback is disappearing, either because the callback
		// ending or the C thread that called it is being destroyed.
		//
		// Also, treat this as if we lost our P too.
		// The thread ID may be reused by the platform and we'll get
		// really confused if we try to steal the P is this is running
		// with later. The new M with the same ID could even try to
		// steal back this P from itself!
		//
		// The runtime is careful to make sure that any GoCreateSyscall
		// event will enter the runtime emitting events for reacquiring a P.
		//
		// Note: we might have a P here. The P might not be released
		// eagerly by the runtime, and it might get stolen back later
		// (or never again, if the program is going to exit).
		if  := validateCtx(, event.SchedReqs{Thread: event.MustHave, Proc: event.MayHave, Goroutine: event.MustHave});  != nil {
			return , false, 
		}
		// Check to make sure the goroutine exists in the right state.
		,  := .gStates[.G]
		if ! {
			return , false, fmt.Errorf("event %s for goroutine (%v) that doesn't exist", go122.EventString(), .G)
		}
		if .status != go122.GoSyscall {
			return , false, fmt.Errorf("%s event for goroutine that's not %v", go122.EventString(), GoSyscall)
		}
		// This goroutine is exiting itself.
		delete(.gStates, .G)
		.G = NoGoroutine

		// If we have a proc, then we're dissociating from it now. See the comment at the top of the case.
		if .P != NoProc {
			,  := .pStates[.P]
			if ! {
				return , false, fmt.Errorf("found invalid proc %d during %s", .P, go122.EventString())
			}
			if .status != go122.ProcSyscall {
				return , false, fmt.Errorf("proc %d in unexpected state %s during %s", .P, .status, go122.EventString())
			}
			// See the go122-create-syscall-reuse-thread-id test case for more details.
			.status = go122.ProcSyscallAbandoned
			.P = NoProc

			// Queue an extra self-ProcSteal event.
			.extraEvent = Event{
				table: ,
				ctx:   ,
				base: baseEvent{
					typ:  go122.EvProcSteal,
					time: .time,
				},
			}
			.extraEvent.base.args[0] = uint64(.P)
			.extraEvent.base.extra(version.Go122)[0] = uint64(go122.ProcSyscall)
		}
		return , true, nil

	// Handle tasks. Tasks are interesting because:
	// - There's no Begin event required to reference a task.
	// - End for a particular task ID can appear multiple times.
	// As a result, there's very little to validate. The only
	// thing we have to be sure of is that a task didn't begin
	// after it had already begun. Task IDs are allowed to be
	// reused, so we don't care about a Begin after an End.
	case go122.EvUserTaskBegin:
		 := TaskID(.args[0])
		if ,  := .activeTasks[];  {
			return , false, fmt.Errorf("task ID conflict: %d", )
		}
		// Get the parent ID, but don't validate it. There's no guarantee
		// we actually have information on whether it's active.
		 := TaskID(.args[1])
		if  == BackgroundTask {
			// Note: a value of 0 here actually means no parent, *not* the
			// background task. Automatic background task attachment only
			// applies to regions.
			 = NoTask
			.args[1] = uint64(NoTask)
		}

		// Validate the name and record it. We'll need to pass it through to
		// EvUserTaskEnd.
		 := stringID(.args[2])
		,  := .strings.get()
		if ! {
			return , false, fmt.Errorf("invalid string ID %v for %v event", , )
		}
		.activeTasks[] = taskState{name: , parentID: }
		return , true, validateCtx(, event.UserGoReqs)
	case go122.EvUserTaskEnd:
		 := TaskID(.args[0])
		if ,  := .activeTasks[];  {
			// Smuggle the task info. This may happen in a different generation,
			// which may not have the name in its string table. Add it to the extra
			// strings table so we can look it up later.
			.extra(version.Go122)[0] = uint64(.parentID)
			.extra(version.Go122)[1] = uint64(.addExtraString(.name))
			delete(.activeTasks, )
		} else {
			// Explicitly clear the task info.
			.extra(version.Go122)[0] = uint64(NoTask)
			.extra(version.Go122)[1] = uint64(.addExtraString(""))
		}
		return , true, validateCtx(, event.UserGoReqs)

	// Handle user regions.
	case go122.EvUserRegionBegin:
		if  := validateCtx(, event.UserGoReqs);  != nil {
			return , false, 
		}
		 := TaskID(.args[0])
		 := stringID(.args[1])
		,  := .strings.get()
		if ! {
			return , false, fmt.Errorf("invalid string ID %v for %v event", , )
		}
		,  := .gStates[.G]
		if ! {
			return , false, fmt.Errorf("encountered EvUserRegionBegin without known state for current goroutine %d", .G)
		}
		if  := .beginRegion(userRegion{, });  != nil {
			return , false, 
		}
		return , true, nil
	case go122.EvUserRegionEnd:
		if  := validateCtx(, event.UserGoReqs);  != nil {
			return , false, 
		}
		 := TaskID(.args[0])
		 := stringID(.args[1])
		,  := .strings.get()
		if ! {
			return , false, fmt.Errorf("invalid string ID %v for %v event", , )
		}
		,  := .gStates[.G]
		if ! {
			return , false, fmt.Errorf("encountered EvUserRegionEnd without known state for current goroutine %d", .G)
		}
		if  := .endRegion(userRegion{, });  != nil {
			return , false, 
		}
		return , true, nil

	// Handle the GC mark phase.
	//
	// We have sequence numbers for both start and end because they
	// can happen on completely different threads. We want an explicit
	// partial order edge between start and end here, otherwise we're
	// relying entirely on timestamps to make sure we don't advance a
	// GCEnd for a _different_ GC cycle if timestamps are wildly broken.
	case go122.EvGCActive:
		 := .args[0]
		if  == .initialGen {
			if .gcState != gcUndetermined {
				return , false, fmt.Errorf("GCActive in the first generation isn't first GC event")
			}
			.gcSeq = 
			.gcState = gcRunning
			return , true, nil
		}
		if  != .gcSeq+1 {
			// This is not the right GC cycle.
			return , false, nil
		}
		if .gcState != gcRunning {
			return , false, fmt.Errorf("encountered GCActive while GC was not in progress")
		}
		.gcSeq = 
		if  := validateCtx(, event.UserGoReqs);  != nil {
			return , false, 
		}
		return , true, nil
	case go122.EvGCBegin:
		 := .args[0]
		if .gcState == gcUndetermined {
			.gcSeq = 
			.gcState = gcRunning
			return , true, nil
		}
		if  != .gcSeq+1 {
			// This is not the right GC cycle.
			return , false, nil
		}
		if .gcState == gcRunning {
			return , false, fmt.Errorf("encountered GCBegin while GC was already in progress")
		}
		.gcSeq = 
		.gcState = gcRunning
		if  := validateCtx(, event.UserGoReqs);  != nil {
			return , false, 
		}
		return , true, nil
	case go122.EvGCEnd:
		 := .args[0]
		if  != .gcSeq+1 {
			// This is not the right GC cycle.
			return , false, nil
		}
		if .gcState == gcNotRunning {
			return , false, fmt.Errorf("encountered GCEnd when GC was not in progress")
		}
		if .gcState == gcUndetermined {
			return , false, fmt.Errorf("encountered GCEnd when GC was in an undetermined state")
		}
		.gcSeq = 
		.gcState = gcNotRunning
		if  := validateCtx(, event.UserGoReqs);  != nil {
			return , false, 
		}
		return , true, nil

	// Handle simple instantaneous events that require a G.
	case go122.EvGoLabel, go122.EvProcsChange, go122.EvUserLog:
		if  := validateCtx(, event.UserGoReqs);  != nil {
			return , false, 
		}
		return , true, nil

	// Handle allocation states, which don't require a G.
	case go122.EvHeapAlloc, go122.EvHeapGoal:
		if  := validateCtx(, event.SchedReqs{Thread: event.MustHave, Proc: event.MustHave, Goroutine: event.MayHave});  != nil {
			return , false, 
		}
		return , true, nil

	// Handle sweep, which is bound to a P and doesn't require a G.
	case go122.EvGCSweepBegin:
		if  := validateCtx(, event.SchedReqs{Thread: event.MustHave, Proc: event.MustHave, Goroutine: event.MayHave});  != nil {
			return , false, 
		}
		if  := .pStates[.P].beginRange(makeRangeType(, 0));  != nil {
			return , false, 
		}
		return , true, nil
	case go122.EvGCSweepActive:
		 := ProcID(.args[0])
		// N.B. In practice Ps can't block while they're sweeping, so this can only
		// ever reference curCtx.P. However, be lenient about this like we are with
		// GCMarkAssistActive; there's no reason the runtime couldn't change to block
		// in the middle of a sweep.
		,  := .pStates[]
		if ! {
			return , false, fmt.Errorf("encountered GCSweepActive for unknown proc %d", )
		}
		if  := .activeRange(makeRangeType(, 0),  == .initialGen);  != nil {
			return , false, 
		}
		return , true, nil
	case go122.EvGCSweepEnd:
		if  := validateCtx(, event.SchedReqs{Thread: event.MustHave, Proc: event.MustHave, Goroutine: event.MayHave});  != nil {
			return , false, 
		}
		,  := .pStates[.P].endRange()
		if  != nil {
			return , false, 
		}
		return , true, nil

	// Handle special goroutine-bound event ranges.
	case go122.EvSTWBegin, go122.EvGCMarkAssistBegin:
		if  := validateCtx(, event.UserGoReqs);  != nil {
			return , false, 
		}
		 := stringID(0)
		if  == go122.EvSTWBegin {
			 = stringID(.args[0])
		}
		,  := .gStates[.G]
		if ! {
			return , false, fmt.Errorf("encountered event of type %d without known state for current goroutine %d", , .G)
		}
		if  := .beginRange(makeRangeType(, ));  != nil {
			return , false, 
		}
		return , true, nil
	case go122.EvGCMarkAssistActive:
		 := GoID(.args[0])
		// N.B. Like GoStatus, this can happen at any time, because it can
		// reference a non-running goroutine. Don't check anything about the
		// current scheduler context.
		,  := .gStates[]
		if ! {
			return , false, fmt.Errorf("uninitialized goroutine %d found during %s", , go122.EventString())
		}
		if  := .activeRange(makeRangeType(, 0),  == .initialGen);  != nil {
			return , false, 
		}
		return , true, nil
	case go122.EvSTWEnd, go122.EvGCMarkAssistEnd:
		if  := validateCtx(, event.UserGoReqs);  != nil {
			return , false, 
		}
		,  := .gStates[.G]
		if ! {
			return , false, fmt.Errorf("encountered event of type %d without known state for current goroutine %d", , .G)
		}
		,  := .endRange()
		if  != nil {
			return , false, 
		}
		if  == go122.EvSTWEnd {
			// Smuggle the kind into the event.
			// Don't use ev.extra here so we have symmetry with STWBegin.
			.args[0] = uint64()
		}
		return , true, nil
	}
	return , false, fmt.Errorf("bad event type found while ordering: %v", .typ)
}

// schedCtx represents the scheduling resources associated with an event.
type schedCtx struct {
	G GoID
	P ProcID
	M ThreadID
}

// validateCtx ensures that ctx conforms to some reqs, returning an error if
// it doesn't.
func validateCtx( schedCtx,  event.SchedReqs) error {
	// Check thread requirements.
	if .Thread == event.MustHave && .M == NoThread {
		return fmt.Errorf("expected a thread but didn't have one")
	} else if .Thread == event.MustNotHave && .M != NoThread {
		return fmt.Errorf("expected no thread but had one")
	}

	// Check proc requirements.
	if .Proc == event.MustHave && .P == NoProc {
		return fmt.Errorf("expected a proc but didn't have one")
	} else if .Proc == event.MustNotHave && .P != NoProc {
		return fmt.Errorf("expected no proc but had one")
	}

	// Check goroutine requirements.
	if .Goroutine == event.MustHave && .G == NoGoroutine {
		return fmt.Errorf("expected a goroutine but didn't have one")
	} else if .Goroutine == event.MustNotHave && .G != NoGoroutine {
		return fmt.Errorf("expected no goroutine but had one")
	}
	return nil
}

// gcState is a trinary variable for the current state of the GC.
//
// The third state besides "enabled" and "disabled" is "undetermined."
type gcState uint8

const (
	gcUndetermined gcState = iota
	gcNotRunning
	gcRunning
)

// String returns a human-readable string for the GC state.
func ( gcState) () string {
	switch  {
	case gcUndetermined:
		return "Undetermined"
	case gcNotRunning:
		return "NotRunning"
	case gcRunning:
		return "Running"
	}
	return "Bad"
}

// userRegion represents a unique user region when attached to some gState.
type userRegion struct {
	// name must be a resolved string because the string ID for the same
	// string may change across generations, but we care about checking
	// the value itself.
	taskID TaskID
	name   string
}

// rangeType is a way to classify special ranges of time.
//
// These typically correspond 1:1 with "Begin" events, but
// they may have an optional subtype that describes the range
// in more detail.
type rangeType struct {
	typ  event.Type // "Begin" event.
	desc stringID   // Optional subtype.
}

// makeRangeType constructs a new rangeType.
func makeRangeType( event.Type,  stringID) rangeType {
	if  := go122.Specs()[].StartEv;  != go122.EvNone {
		 = 
	}
	return rangeType{, }
}

// gState is the state of a goroutine at a point in the trace.
type gState struct {
	id     GoID
	status go122.GoStatus
	seq    seqCounter

	// regions are the active user regions for this goroutine.
	regions []userRegion

	// rangeState is the state of special time ranges bound to this goroutine.
	rangeState
}

// beginRegion starts a user region on the goroutine.
func ( *gState) ( userRegion) error {
	.regions = append(.regions, )
	return nil
}

// endRegion ends a user region on the goroutine.
func ( *gState) ( userRegion) error {
	if len(.regions) == 0 {
		// We do not know about regions that began before tracing started.
		return nil
	}
	if  := .regions[len(.regions)-1];  !=  {
		return fmt.Errorf("misuse of region in goroutine %v: region end %v when the inner-most active region start event is %v", .id, , )
	}
	.regions = .regions[:len(.regions)-1]
	return nil
}

// pState is the state of a proc at a point in the trace.
type pState struct {
	id     ProcID
	status go122.ProcStatus
	seq    seqCounter

	// rangeState is the state of special time ranges bound to this proc.
	rangeState
}

// mState is the state of a thread at a point in the trace.
type mState struct {
	g GoID   // Goroutine bound to this M. (The goroutine's state is Executing.)
	p ProcID // Proc bound to this M. (The proc's state is Executing.)
}

// rangeState represents the state of special time ranges.
type rangeState struct {
	// inFlight contains the rangeTypes of any ranges bound to a resource.
	inFlight []rangeType
}

// beginRange begins a special range in time on the goroutine.
//
// Returns an error if the range is already in progress.
func ( *rangeState) ( rangeType) error {
	if .hasRange() {
		return fmt.Errorf("discovered event already in-flight for when starting event %v", go122.Specs()[.typ].Name)
	}
	.inFlight = append(.inFlight, )
	return nil
}

// activeRange marks special range in time on the goroutine as active in the
// initial generation, or confirms that it is indeed active in later generations.
func ( *rangeState) ( rangeType,  bool) error {
	if  {
		if .hasRange() {
			return fmt.Errorf("found named active range already in first gen: %v", )
		}
		.inFlight = append(.inFlight, )
	} else if !.hasRange() {
		return fmt.Errorf("resource is missing active range: %v %v", go122.Specs()[.typ].Name, .inFlight)
	}
	return nil
}

// hasRange returns true if a special time range on the goroutine as in progress.
func ( *rangeState) ( rangeType) bool {
	for ,  := range .inFlight {
		if  ==  {
			return true
		}
	}
	return false
}

// endsRange ends a special range in time on the goroutine.
//
// This must line up with the start event type  of the range the goroutine is currently in.
func ( *rangeState) ( event.Type) (stringID, error) {
	 := go122.Specs()[].StartEv
	 := -1
	for ,  := range .inFlight {
		if .typ ==  {
			 = 
			break
		}
	}
	if  < 0 {
		return 0, fmt.Errorf("tried to end event %v, but not in-flight", go122.Specs()[].Name)
	}
	// Swap remove.
	 := .inFlight[].desc
	.inFlight[], .inFlight[len(.inFlight)-1] = .inFlight[len(.inFlight)-1], .inFlight[]
	.inFlight = .inFlight[:len(.inFlight)-1]
	return , nil
}

// seqCounter represents a global sequence counter for a resource.
type seqCounter struct {
	gen uint64 // The generation for the local sequence counter seq.
	seq uint64 // The sequence number local to the generation.
}

// makeSeq creates a new seqCounter.
func makeSeq(,  uint64) seqCounter {
	return seqCounter{gen: , seq: }
}

// succeeds returns true if a is the immediate successor of b.
func ( seqCounter) ( seqCounter) bool {
	return .gen == .gen && .seq == .seq+1
}

// String returns a debug string representation of the seqCounter.
func ( seqCounter) () string {
	return fmt.Sprintf("%d (gen=%d)", .seq, .gen)
}

func dumpOrdering( *ordering) string {
	var  strings.Builder
	for ,  := range .gStates {
		fmt.Fprintf(&, "G %d [status=%s seq=%s]\n", , .status, .seq)
	}
	fmt.Fprintln(&)
	for ,  := range .pStates {
		fmt.Fprintf(&, "P %d [status=%s seq=%s]\n", , .status, .seq)
	}
	fmt.Fprintln(&)
	for ,  := range .mStates {
		fmt.Fprintf(&, "M %d [g=%d p=%d]\n", , .g, .p)
	}
	fmt.Fprintln(&)
	fmt.Fprintf(&, "GC %d %s\n", .gcSeq, .gcState)
	return .String()
}

// taskState represents an active task.
type taskState struct {
	// name is the type of the active task.
	name string

	// parentID is the parent ID of the active task.
	parentID TaskID
}