service/dap: support running requests asynchronously (#2423)

* service/dap: refine teardown logic

* Address review comments + add missing lock/unlock

* Narrow lock scope

* Update comments only

* Remove redundan temp var from stopNoDebugProcess

* Clarify comment

* service/dap: support running requests asynchronously

* Respond to review comments

* Remove debugging printf

* Refine locking. Add call TODO. Use -1 for running thread id.

* Fix TestAttachStopOnEntry: it can terminate on halt

* Respond to review comments

* Recover panics in async requests handling

* defer releasing asyncSetupDone, so it happens on panic

Co-authored-by: Polina Sokolova <polinasok@users.noreply.github.com>
This commit is contained in:
polinasok 2021-05-04 12:49:52 -07:00 committed by GitHub
parent 0da1c9a9d7
commit 03f1ec1dfc
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
4 changed files with 391 additions and 164 deletions

@ -20,6 +20,7 @@ const (
UnableToListGlobals = 2007
UnableToLookupVariable = 2008
UnableToEvaluateExpression = 2009
// Add more codes as we support more requests
DisconnectError = 5000
DebuggeeIsRunning = 4000
DisconnectError = 5000
)

@ -37,7 +37,7 @@ import (
)
// Server implements a DAP server that can accept a single client for
// a single debug session (for now). It does yet not support restarting.
// a single debug session (for now). It does not yet support restarting.
// That means that in addition to explicit shutdown requests,
// program termination and failed or closed client connection
// would also result in stopping this single-use server.
@ -53,21 +53,41 @@ import (
//
// (2) Run goroutine started from Run() that serves as both
// a listener and a client goroutine. It accepts a client connection,
// reads, decodes and processes each request, issuing commands to the
// underlying debugger and sending back events and responses.
// This gorouitne sends a stop-server signal via config.DisconnecChan
// when encounering a client connection error or responding to
// a DAP disconnect request.
// reads, decodes and dispatches each request from the client.
// For synchronous requests, it issues commands to the
// underlying debugger and sends back events and responses.
// These requests block while the debuggee is running, so,
// where applicable, the handlers need to check if debugging
// state is running, so there is a need for a halt request or
// a dummy/error response to avoid blocking.
//
// This is the only goroutine that sends a stop-server signal
// via config.DisconnecChan when encountering a client connection
// error or responding to a (synchronous) DAP disconnect request.
// Once stop is triggered, the goroutine exits.
//
// TODO(polina): add another layer of per-client goroutines to support multiple clients
// TODO(polina): make it asynchronous (i.e. launch goroutine per request)
//
// (3) Per-request goroutine is started for each asynchronous request
// that resumes execution. We check if target is running already, so
// there should be no more than one pending asynchronous request at
// a time. This goroutine issues commands to the underlying debugger
// and sends back events and responses. It takes a setup-done channel
// as an argument and temporarily blocks the request loop until setup
// for asynchronous execution is complete and targe is running.
// Once done, it unblocks processing of parallel requests unblocks
// (e.g. disconnecting while the program is running).
//
// These per-request goroutines never send a stop-server signal.
// They block on running debugger commands that are interrupted
// when halt is issued while stopping. At that point these goroutines
// wrap-up and exit.
type Server struct {
// config is all the information necessary to start the debugger and server.
config *service.Config
// listener is used to accept the client connection.
listener net.Listener
// stopTriggered is closed when the server is Stop()-ed. This can be used to signal
// to goroutines run by the server that it's time to quit.
// stopTriggered is closed when the server is Stop()-ed.
stopTriggered chan struct{}
// reader is used to read requests from the connection.
reader *bufio.Reader
@ -86,6 +106,7 @@ type Server struct {
// mu synchronizes access to objects set on start-up (from run goroutine)
// and stopped on teardown (from main goroutine)
mu sync.Mutex
// conn is the accepted client connection.
conn net.Conn
// debugger is the underlying debugger service.
@ -94,6 +115,10 @@ type Server struct {
binaryToRemove string
// noDebugProcess is set for the noDebug launch process.
noDebugProcess *exec.Cmd
// sendingMu synchronizes writing to net.Conn
// to ensure that messages do not get interleaved
sendingMu sync.Mutex
}
// launchAttachArgs captures arguments from launch/attach request that
@ -217,6 +242,7 @@ func (s *Server) Stop() {
// allowing the run goroutine to exit.
_ = s.conn.Close()
}
if s.debugger != nil {
killProcess := s.config.Debugger.AttachPid == 0
s.stopDebugSession(killProcess)
@ -238,8 +264,6 @@ func (s *Server) Stop() {
// The function safeguards agaist closing the channel more
// than once and can be called multiple times. It is not thread-safe
// and is currently only called from the run goroutine.
// TODO(polina): lock this when we add more goroutines that could call
// this when we support asynchronous request-response communication.
func (s *Server) triggerServerStop() {
// Avoid accidentally closing the channel twice and causing a panic, when
// this function is called more than once. For example, we could have the
@ -314,20 +338,146 @@ func (s *Server) serveDAPCodec() {
}
}
// In case a handler panics, we catch the panic to avoid crashing both
// the server and the target. We send an error response back, but
// in case its a dup and ignored by the client, we also log the error.
func (s *Server) recoverPanic(request dap.Message) {
if ierr := recover(); ierr != nil {
s.log.Errorf("recovered panic: %s\n%s\n", ierr, debug.Stack())
s.sendInternalErrorResponse(request.GetSeq(), fmt.Sprintf("%v", ierr))
}
}
func (s *Server) handleRequest(request dap.Message) {
defer func() {
// In case a handler panics, we catch the panic and send an error response
// back to the client.
if ierr := recover(); ierr != nil {
s.log.Errorf("stacktrace from recovered panic:\n%s\n", debug.Stack())
s.sendInternalErrorResponse(request.GetSeq(), fmt.Sprintf("%v", ierr))
}
}()
defer s.recoverPanic(request)
jsonmsg, _ := json.Marshal(request)
s.log.Debug("[<- from client]", string(jsonmsg))
if _, ok := request.(dap.RequestMessage); !ok {
s.sendInternalErrorResponse(request.GetSeq(), fmt.Sprintf("Unable to process non-request %#v\n", request))
return
}
// These requests, can be handeled regardless of whether the targret is running
switch request := request.(type) {
case *dap.DisconnectRequest:
// Required
s.onDisconnectRequest(request)
return
case *dap.PauseRequest:
// Required
// TODO: implement this request in V0
s.onPauseRequest(request)
return
case *dap.TerminateRequest:
// Optional (capability supportsTerminateRequest)
// TODO: implement this request in V1
s.onTerminateRequest(request)
return
case *dap.RestartRequest:
// Optional (capability supportsRestartRequest)
// TODO: implement this request in V1
s.onRestartRequest(request)
return
}
// Most requests cannot be processed while the debuggee is running.
// We have a couple of options for handling these without blocking
// the request loop indefinitely when we are in running state.
// --1-- Return a dummy response or an error right away.
// --2-- Halt execution, process the request, resume execution.
// TODO(polina): do this for setting breakpoints
// --3-- Handle such requests asynchronously and let them block until
// the process stops or terminates (e.g. using a channel and a single
// goroutine to preserve the order). This might not be appropriate
// for requests such as continue or step because they would skip
// the stop, resuming execution right away. Other requests
// might not be relevant anymore when the stop is finally reached, and
// state changed from the previous snapshot. The user might want to
// resume execution before the backlog of buffered requests is cleared,
// so we would have to either cancel them or delay processing until
// the next stop. In addition, the editor itself might block waiting
// for these requests to return. We are not aware of any requests
// that would benefit from this approach at this time.
if s.debugger != nil && s.debugger.IsRunning() {
switch request := request.(type) {
case *dap.ThreadsRequest:
// On start-up, the client requests the baseline of currently existing threads
// right away as there are a number of DAP requests that require a thread id
// (pause, continue, stacktrace, etc). This can happen after the program
// continues on entry, preventing the client from handling any pause requests
// from the user. We remedy this by sending back a placeholder thread id
// for the current goroutine.
response := &dap.ThreadsResponse{
Response: *newResponse(request.Request),
Body: dap.ThreadsResponseBody{Threads: []dap.Thread{{Id: -1, Name: "Current"}}},
}
s.send(response)
default:
r := request.(dap.RequestMessage).GetRequest()
s.sendErrorResponse(*r, DebuggeeIsRunning, fmt.Sprintf("Unable to process `%s`", r.Command), "debuggee is running")
}
return
}
// Requests below can only be handled while target is stopped.
// Some of them are blocking and will be handled synchronously
// on this goroutine while non-blocking requests will be dispatched
// to another goroutine. Please note that because of the running
// check above, there should be no more than one pending asynchronous
// request at a time.
// Non-blocking request handlers will signal when they are ready
// setting up for async execution, so more requests can be processed.
resumeRequestLoop := make(chan struct{})
switch request := request.(type) {
//--- Asynchronous requests ---
case *dap.ConfigurationDoneRequest:
// Optional (capability supportsConfigurationDoneRequest)
go func() {
defer s.recoverPanic(request)
s.onConfigurationDoneRequest(request, resumeRequestLoop)
}()
<-resumeRequestLoop
case *dap.ContinueRequest:
// Required
go func() {
defer s.recoverPanic(request)
s.onContinueRequest(request, resumeRequestLoop)
}()
<-resumeRequestLoop
case *dap.NextRequest:
// Required
go func() {
defer s.recoverPanic(request)
s.onNextRequest(request, resumeRequestLoop)
}()
<-resumeRequestLoop
case *dap.StepInRequest:
// Required
go func() {
defer s.recoverPanic(request)
s.onStepInRequest(request, resumeRequestLoop)
}()
<-resumeRequestLoop
case *dap.StepOutRequest:
// Required
go func() {
defer s.recoverPanic(request)
s.onStepOutRequest(request, resumeRequestLoop)
}()
<-resumeRequestLoop
case *dap.StepBackRequest:
// Optional (capability supportsStepBack)
// TODO: implement this request in V1
s.onStepBackRequest(request)
case *dap.ReverseContinueRequest:
// Optional (capability supportsStepBack)
// TODO: implement this request in V1
s.onReverseContinueRequest(request)
//--- Synchronous requests ---
case *dap.InitializeRequest:
// Required
s.onInitializeRequest(request)
@ -337,17 +487,6 @@ func (s *Server) handleRequest(request dap.Message) {
case *dap.AttachRequest:
// Required
s.onAttachRequest(request)
case *dap.DisconnectRequest:
// Required
s.onDisconnectRequest(request)
case *dap.TerminateRequest:
// Optional (capability supportsTerminateRequest)
// TODO: implement this request in V1
s.onTerminateRequest(request)
case *dap.RestartRequest:
// Optional (capability supportsRestartRequest)
// TODO: implement this request in V1
s.onRestartRequest(request)
case *dap.SetBreakpointsRequest:
// Required
s.onSetBreakpointsRequest(request)
@ -358,40 +497,9 @@ func (s *Server) handleRequest(request dap.Message) {
case *dap.SetExceptionBreakpointsRequest:
// Optional (capability exceptionBreakpointFilters)
s.onSetExceptionBreakpointsRequest(request)
case *dap.ConfigurationDoneRequest:
// Optional (capability supportsConfigurationDoneRequest)
// Supported by vscode-go
s.onConfigurationDoneRequest(request)
case *dap.ContinueRequest:
case *dap.ThreadsRequest:
// Required
s.onContinueRequest(request)
case *dap.NextRequest:
// Required
s.onNextRequest(request)
case *dap.StepInRequest:
// Required
s.onStepInRequest(request)
case *dap.StepOutRequest:
// Required
s.onStepOutRequest(request)
case *dap.StepBackRequest:
// Optional (capability supportsStepBack)
// TODO: implement this request in V1
s.onStepBackRequest(request)
case *dap.ReverseContinueRequest:
// Optional (capability supportsStepBack)
// TODO: implement this request in V1
s.onReverseContinueRequest(request)
case *dap.RestartFrameRequest:
// Optional (capability supportsRestartFrame)
s.sendUnsupportedErrorResponse(request.Request)
case *dap.GotoRequest:
// Optional (capability supportsGotoTargetsRequest)
s.sendUnsupportedErrorResponse(request.Request)
case *dap.PauseRequest:
// Required
// TODO: implement this request in V0
s.onPauseRequest(request)
s.onThreadsRequest(request)
case *dap.StackTraceRequest:
// Required
s.onStackTraceRequest(request)
@ -401,6 +509,9 @@ func (s *Server) handleRequest(request dap.Message) {
case *dap.VariablesRequest:
// Required
s.onVariablesRequest(request)
case *dap.EvaluateRequest:
// Required
s.onEvaluateRequest(request)
case *dap.SetVariableRequest:
// Optional (capability supportsSetVariable)
// Supported by vscode-go
@ -410,20 +521,37 @@ func (s *Server) handleRequest(request dap.Message) {
// Optional (capability supportsSetExpression)
// TODO: implement this request in V1
s.onSetExpressionRequest(request)
case *dap.LoadedSourcesRequest:
// Optional (capability supportsLoadedSourcesRequest)
// TODO: implement this request in V1
s.onLoadedSourcesRequest(request)
case *dap.ReadMemoryRequest:
// Optional (capability supportsReadMemoryRequest)
// TODO: implement this request in V1
s.onReadMemoryRequest(request)
case *dap.DisassembleRequest:
// Optional (capability supportsDisassembleRequest)
// TODO: implement this request in V1
s.onDisassembleRequest(request)
case *dap.CancelRequest:
// Optional (capability supportsCancelRequest)
// TODO: does this request make sense for delve?
s.onCancelRequest(request)
//--- Requests that we do not plan to support ---
case *dap.RestartFrameRequest:
// Optional (capability supportsRestartFrame)
s.sendUnsupportedErrorResponse(request.Request)
case *dap.GotoRequest:
// Optional (capability supportsGotoTargetsRequest)
s.sendUnsupportedErrorResponse(request.Request)
case *dap.SourceRequest:
// Required
// This does not make sense in the context of Go as
// the source cannot be a string eval'ed at runtime.
s.sendUnsupportedErrorResponse(request.Request)
case *dap.ThreadsRequest:
// Required
s.onThreadsRequest(request)
case *dap.TerminateThreadsRequest:
// Optional (capability supportsTerminateThreadsRequest)
s.sendUnsupportedErrorResponse(request.Request)
case *dap.EvaluateRequest:
// Required
s.onEvaluateRequest(request)
case *dap.StepInTargetsRequest:
// Optional (capability supportsStepInTargetsRequest)
s.sendUnsupportedErrorResponse(request.Request)
@ -437,28 +565,12 @@ func (s *Server) handleRequest(request dap.Message) {
// Optional (capability supportsExceptionInfoRequest)
// TODO: does this request make sense for delve?
s.sendUnsupportedErrorResponse(request.Request)
case *dap.LoadedSourcesRequest:
// Optional (capability supportsLoadedSourcesRequest)
// TODO: implement this request in V1
s.onLoadedSourcesRequest(request)
case *dap.DataBreakpointInfoRequest:
// Optional (capability supportsDataBreakpoints)
s.sendUnsupportedErrorResponse(request.Request)
case *dap.SetDataBreakpointsRequest:
// Optional (capability supportsDataBreakpoints)
s.sendUnsupportedErrorResponse(request.Request)
case *dap.ReadMemoryRequest:
// Optional (capability supportsReadMemoryRequest)
// TODO: implement this request in V1
s.onReadMemoryRequest(request)
case *dap.DisassembleRequest:
// Optional (capability supportsDisassembleRequest)
// TODO: implement this request in V1
s.onDisassembleRequest(request)
case *dap.CancelRequest:
// Optional (capability supportsCancelRequest)
// TODO: does this request make sense for delve?
s.onCancelRequest(request)
case *dap.BreakpointLocationsRequest:
// Optional (capability supportsBreakpointLocationsRequest)
s.sendUnsupportedErrorResponse(request.Request)
@ -477,6 +589,11 @@ func (s *Server) handleRequest(request dap.Message) {
func (s *Server) send(message dap.Message) {
jsonmsg, _ := json.Marshal(message)
s.log.Debug("[-> to client]", string(jsonmsg))
// TODO(polina): consider using a channel for all the sends and to have a dedicated
// goroutine that reads from that channel and sends over the connection.
// This will avoid blocking on slow network sends.
s.sendingMu.Lock()
defer s.sendingMu.Unlock()
_ = dap.WriteProtocolMessage(s.conn, message)
}
@ -757,9 +874,15 @@ func (s *Server) stopDebugSession(killProcess bool) error {
}
var err error
var exited error
// Halting will stop any debugger command that's pending on another
// per-request goroutine, hence unblocking that goroutine to wrap-up and exit.
// TODO(polina): Per-request goroutine could still not be done when this one is.
// To avoid goroutine leaks, we can use a wait group or have the goroutine listen
// for a stop signal on a dedicated quit channel at suitable points (use context?).
// Additional clean-up might be especially critical when we support multiple clients.
state, err := s.debugger.Command(&api.DebuggerCommand{Name: api.Halt}, nil)
if err == proc.ErrProcessDetached {
s.log.Debug(err)
s.log.Debug("halt returned error:", err)
return nil
}
if err != nil {
@ -767,10 +890,11 @@ func (s *Server) stopDebugSession(killProcess bool) error {
case proc.ErrProcessExited:
exited = err
default:
s.log.Error(err)
s.log.Error("halt returned error:", err)
}
} else if state.Exited {
exited = proc.ErrProcessExited{Pid: s.debugger.ProcessPid(), Status: state.ExitStatus}
s.log.Debug("halt returned state:", exited)
}
if exited != nil {
s.logToConsole(exited.Error())
@ -795,12 +919,17 @@ func (s *Server) stopDebugSession(killProcess bool) error {
return err
}
func (s *Server) isNoDebug() bool {
s.mu.Lock()
defer s.mu.Unlock()
return s.noDebugProcess != nil
}
func (s *Server) onSetBreakpointsRequest(request *dap.SetBreakpointsRequest) {
if s.noDebugProcess != nil {
if s.isNoDebug() {
s.sendErrorResponse(request.Request, UnableToSetBreakpoints, "Unable to set or clear breakpoints", "running in noDebug mode")
return
}
// TODO(polina): handle this while running by halting first.
if request.Arguments.Source.Path == "" {
s.sendErrorResponse(request.Request, UnableToSetBreakpoints, "Unable to set or clear breakpoints", "empty file path")
@ -865,8 +994,24 @@ func (s *Server) onSetExceptionBreakpointsRequest(request *dap.SetExceptionBreak
s.send(&dap.SetExceptionBreakpointsResponse{Response: *newResponse(request.Request)})
}
func (s *Server) onConfigurationDoneRequest(request *dap.ConfigurationDoneRequest) {
if s.debugger != nil && s.args.stopOnEntry {
func (s *Server) asyncCommandDone(asyncSetupDone chan struct{}) {
if asyncSetupDone != nil {
select {
case <-asyncSetupDone:
// already closed
default:
close(asyncSetupDone)
}
}
}
// onConfigurationDoneRequest handles 'configurationDone' request.
// This is an optional request enabled by capability supportsConfigurationDoneRequest.
// It gets triggered after all the debug requests that followinitalized event,
// so the s.debugger is guaranteed to be set.
func (s *Server) onConfigurationDoneRequest(request *dap.ConfigurationDoneRequest, asyncSetupDone chan struct{}) {
defer s.asyncCommandDone(asyncSetupDone)
if s.args.stopOnEntry {
e := &dap.StoppedEvent{
Event: *newEvent("stopped"),
Body: dap.StoppedEventBody{Reason: "entry", ThreadId: 1, AllThreadsStopped: true},
@ -874,16 +1019,18 @@ func (s *Server) onConfigurationDoneRequest(request *dap.ConfigurationDoneReques
s.send(e)
}
s.send(&dap.ConfigurationDoneResponse{Response: *newResponse(request.Request)})
if s.debugger != nil && !s.args.stopOnEntry {
s.doCommand(api.Continue)
if !s.args.stopOnEntry {
s.doCommand(api.Continue, asyncSetupDone)
}
}
func (s *Server) onContinueRequest(request *dap.ContinueRequest) {
// onContinueRequest handles 'continue' request.
// This is a mandatory request to support.
func (s *Server) onContinueRequest(request *dap.ContinueRequest, asyncSetupDone chan struct{}) {
s.send(&dap.ContinueResponse{
Response: *newResponse(request.Request),
Body: dap.ContinueResponseBody{AllThreadsContinued: true}})
s.doCommand(api.Continue)
s.doCommand(api.Continue, asyncSetupDone)
}
func fnName(loc *proc.Location) string {
@ -893,11 +1040,15 @@ func fnName(loc *proc.Location) string {
return loc.Fn.Name
}
// onThreadsRequest handles 'threads' request.
// This is a mandatory request to support.
// It is sent in response to configurationDone response and stopped events.
func (s *Server) onThreadsRequest(request *dap.ThreadsRequest) {
if s.debugger == nil {
s.sendErrorResponse(request.Request, UnableToDisplayThreads, "Unable to display threads", "debugger is nil")
return
}
gs, _, err := s.debugger.Goroutines(0, 0)
if err != nil {
switch err.(type) {
@ -997,23 +1148,23 @@ func (s *Server) onAttachRequest(request *dap.AttachRequest) {
// onNextRequest handles 'next' request.
// This is a mandatory request to support.
func (s *Server) onNextRequest(request *dap.NextRequest) {
func (s *Server) onNextRequest(request *dap.NextRequest, asyncSetupDone chan struct{}) {
s.send(&dap.NextResponse{Response: *newResponse(request.Request)})
s.doStepCommand(api.Next, request.Arguments.ThreadId)
s.doStepCommand(api.Next, request.Arguments.ThreadId, asyncSetupDone)
}
// onStepInRequest handles 'stepIn' request
// This is a mandatory request to support.
func (s *Server) onStepInRequest(request *dap.StepInRequest) {
func (s *Server) onStepInRequest(request *dap.StepInRequest, asyncSetupDone chan struct{}) {
s.send(&dap.StepInResponse{Response: *newResponse(request.Request)})
s.doStepCommand(api.Step, request.Arguments.ThreadId)
s.doStepCommand(api.Step, request.Arguments.ThreadId, asyncSetupDone)
}
// onStepOutRequest handles 'stepOut' request
// This is a mandatory request to support.
func (s *Server) onStepOutRequest(request *dap.StepOutRequest) {
func (s *Server) onStepOutRequest(request *dap.StepOutRequest, asyncSetupDone chan struct{}) {
s.send(&dap.StepOutResponse{Response: *newResponse(request.Request)})
s.doStepCommand(api.StepOut, request.Arguments.ThreadId)
s.doStepCommand(api.StepOut, request.Arguments.ThreadId, asyncSetupDone)
}
func stoppedGoroutineID(state *api.DebuggerState) (id int) {
@ -1025,8 +1176,13 @@ func stoppedGoroutineID(state *api.DebuggerState) (id int) {
return id
}
func (s *Server) doStepCommand(command string, threadId int) {
// Use SwitchGoroutine to change the current goroutine.
// doStepCommand is a wrapper around doCommand that
// first switches selected goroutine. asyncSetupDone is
// a channel that will be closed to signal that an
// asynchornous command has completed setup or was interrupted
// due to an error, so the server is ready to receive new requests.
func (s *Server) doStepCommand(command string, threadId int, asyncSetupDone chan struct{}) {
defer s.asyncCommandDone(asyncSetupDone)
_, err := s.debugger.Command(&api.DebuggerCommand{Name: api.SwitchGoroutine, GoroutineID: threadId}, nil)
if err != nil {
s.log.Errorf("Error switching goroutines while stepping: %v", err)
@ -1044,7 +1200,7 @@ func (s *Server) doStepCommand(command string, threadId int) {
s.send(stopped)
return
}
s.doCommand(command)
s.doCommand(command, asyncSetupDone)
}
// onPauseRequest sends a not-yet-implemented error response.
@ -1062,6 +1218,8 @@ type stackFrame struct {
// onStackTraceRequest handles stackTrace requests.
// This is a mandatory request to support.
// As per DAP spec, this request only gets triggered as a follow-up
// to a successful threads request as part of the "request waterfall".
func (s *Server) onStackTraceRequest(request *dap.StackTraceRequest) {
goroutineID := request.Arguments.ThreadId
frames, err := s.debugger.Stacktrace(goroutineID, s.args.stackTraceDepth, 0)
@ -1101,6 +1259,9 @@ func (s *Server) onStackTraceRequest(request *dap.StackTraceRequest) {
// onScopesRequest handles 'scopes' requests.
// This is a mandatory request to support.
// It is automatically sent as part of the threads > stacktrace > scopes > variables
// "waterfall" to highlight the topmost frame at stops, after an evaluate request
// for the selected scope or when a user selects different scopes in the UI.
func (s *Server) onScopesRequest(request *dap.ScopesRequest) {
sf, ok := s.stackFrameHandles.get(request.Arguments.FrameId)
if !ok {
@ -1431,6 +1592,7 @@ func (s *Server) onEvaluateRequest(request *dap.EvaluateRequest) {
s.sendErrorResponseWithOpts(request.Request, UnableToEvaluateExpression, "Unable to evaluate expression", "debugger is nil", showErrorToUser)
return
}
// Default to the topmost stack frame of the current goroutine in case
// no frame is specified (e.g. when stopped on entry or no call stack frame is expanded)
goid, frame := -1, 0
@ -1457,6 +1619,9 @@ func (s *Server) onEvaluateRequest(request *dap.EvaluateRequest) {
s.sendErrorResponseWithOpts(request.Request, UnableToEvaluateExpression, "Unable to evaluate expression", err.Error(), showErrorToUser)
return
}
// TODO(polina): since call will resume execution of all goroutines,
// we should do this asynchronously and send a continued event to the
// editor, followed by a stop event when the call completes.
state, err := s.debugger.Command(&api.DebuggerCommand{
Name: api.Call,
ReturnInfoLoadConfig: api.LoadConfigFromProc(&DefaultLoadConfig),
@ -1677,16 +1842,18 @@ func (s *Server) resetHandlesForStoppedEvent() {
// doCommand runs a debugger command until it stops on
// termination, error, breakpoint, etc, when an appropriate
// event needs to be sent to the client.
func (s *Server) doCommand(command string) {
if s.debugger == nil {
return
}
state, err := s.debugger.Command(&api.DebuggerCommand{Name: command}, nil)
// event needs to be sent to the client. asyncSetupDone is
// a channel that will be closed to signal that an
// asynchornous command has completed setup or was interrupted
// due to an error, so the server is ready to receive new requests.
func (s *Server) doCommand(command string, asyncSetupDone chan struct{}) {
// TODO(polina): it appears that debugger.Command doesn't close
// asyncSetupDone (e.g. when having an error next while nexting).
// So we should always close it ourselves just in case.
defer s.asyncCommandDone(asyncSetupDone)
state, err := s.debugger.Command(&api.DebuggerCommand{Name: command}, asyncSetupDone)
if _, isexited := err.(proc.ErrProcessExited); isexited || err == nil && state.Exited {
e := &dap.TerminatedEvent{Event: *newEvent("terminated")}
s.send(e)
s.send(&dap.TerminatedEvent{Event: *newEvent("terminated")})
return
}
@ -1697,9 +1864,15 @@ func (s *Server) doCommand(command string) {
if err == nil {
stopped.Body.ThreadId = stoppedGoroutineID(state)
switch s.debugger.StopReason() {
sr := s.debugger.StopReason()
s.log.Debugf("%q command stopped - reason %q", command, sr)
switch sr {
case proc.StopNextFinished:
stopped.Body.Reason = "step"
case proc.StopManual: // triggered by halt
stopped.Body.Reason = "pause"
case proc.StopUnknown: // can happen while stopping
stopped.Body.Reason = "unknown"
default:
stopped.Body.Reason = "breakpoint"
}
@ -1711,7 +1884,6 @@ func (s *Server) doCommand(command string) {
stopped.Body.Reason = "panic"
}
}
s.send(stopped)
} else {
s.log.Error("runtime error: ", err)
stopped.Body.Reason = "runtime error"
@ -1724,7 +1896,6 @@ func (s *Server) doCommand(command string) {
if err == nil {
stopped.Body.ThreadId = stoppedGoroutineID(state)
}
s.send(stopped)
// TODO(polina): according to the spec, the extra 'text' is supposed to show up in the UI (e.g. on hover),
// but so far I am unable to get this to work in vscode - see https://github.com/microsoft/vscode/issues/104475.
@ -1739,6 +1910,11 @@ func (s *Server) doCommand(command string) {
Category: "stderr",
}})
}
// NOTE: If we happen to be responding to another request with an is-running
// error while this one completes, it is possible that the error response
// will arrive after this stopped event.
s.send(stopped)
}
func (s *Server) toClientPath(path string) string {

@ -341,22 +341,51 @@ func TestAttachStopOnEntry(t *testing.T) {
t.Errorf("\ngot %#v\nwant Seq=0, RequestSeq=10 Result=2", evResp)
}
// We cannot contiue here like in the launch case becase the program
// will never terminate. Since we won't receive a terminate event and
// there are no breakpoints to stop on, we just detach.
// 12 >> continue, << continue
client.ContinueRequest(1)
cResp := client.ExpectContinueResponse(t)
if cResp.Seq != 0 || cResp.RequestSeq != 12 {
t.Errorf("\ngot %#v\nwant Seq=0, RequestSeq=12", cResp)
}
// TODO(polina): once https://github.com/go-delve/delve/issues/2259 is
// fixed, test with kill=false.
// 12 >> disconnect, << disconnect
// 13 >> disconnect, << disconnect
client.DisconnectRequestWithKillOption(true)
oed := client.ExpectOutputEventDetachingKill(t)
if oed.Seq != 0 || oed.Body.Category != "console" {
t.Errorf("\ngot %#v\nwant Seq=0 Category='console'", oed)
// Both of these scenarios are somehow possible.
// Even though the program has an infininte loop,
// it apears that a halt can cause it to terminate.
// Since we are in async mode while running, we might receive messages in either order.
msg := client.ExpectMessage(t)
switch m := msg.(type) {
case *dap.StoppedEvent:
if m.Seq != 0 || m.Body.Reason != "pause" { // continue is interrupted
t.Errorf("\ngot %#v\nwant Seq=0 Reason='pause'", m)
}
oed := client.ExpectOutputEventDetachingKill(t)
if oed.Seq != 0 || oed.Body.Category != "console" {
t.Errorf("\ngot %#v\nwant Seq=0 Category='console'", oed)
}
case *dap.TerminatedEvent:
if m.Seq != 0 {
t.Errorf("\ngot %#v\nwant Seq=0'", m)
}
oep := client.ExpectOutputEventProcessExited(t, 0)
if oep.Seq != 0 || oep.Body.Category != "console" {
t.Errorf("\ngot %#v\nwant Seq=0 Category='console'", oep)
}
oed := client.ExpectOutputEventDetaching(t)
if oed.Seq != 0 || oed.Body.Category != "console" {
t.Errorf("\ngot %#v\nwant Seq=0 Category='console'", oed)
}
default:
t.Fatalf("got %#v, want StoppedEvent or TerminatedEvent", m)
}
dResp := client.ExpectDisconnectResponse(t)
if dResp.Seq != 0 || dResp.RequestSeq != 12 {
t.Errorf("\ngot %#v\nwant Seq=0, RequestSeq=12", dResp)
if dResp.Seq != 0 || dResp.RequestSeq != 13 {
t.Errorf("\ngot %#v\nwant Seq=0, RequestSeq=13", dResp)
}
})
}
@ -384,35 +413,47 @@ func TestContinueOnEntry(t *testing.T) {
// 5 >> configurationDone, << configurationDone
client.ConfigurationDoneRequest()
client.ExpectConfigurationDoneResponse(t)
// "Continue" happens behind the scenes
// For now continue is blocking and runs until a stop or
// termination. But once we upgrade the server to be async,
// a simultaneous threads request can be made while continue
// is running. Note that vscode-go just keeps track of the
// continue state and would just return a dummy response
// without talking to debugger if continue was in progress.
// TODO(polina): test this once it is possible
client.ExpectTerminatedEvent(t)
// It is possible for the program to terminate before the initial
// threads request is processed.
// "Continue" happens behind the scenes on another goroutine
// 6 >> threads, << threads
client.ThreadsRequest()
tResp := client.ExpectThreadsResponse(t)
if tResp.Seq != 0 || tResp.RequestSeq != 6 || len(tResp.Body.Threads) != 0 {
t.Errorf("\ngot %#v\nwant Seq=0, RequestSeq=6 len(Threads)=0", tResp)
// Since we are in async mode while running, we might receive messages in either order.
for i := 0; i < 2; i++ {
msg := client.ExpectMessage(t)
switch m := msg.(type) {
case *dap.ThreadsResponse:
if m.Seq != 0 || m.RequestSeq != 6 {
t.Errorf("\ngot %#v\nwant Seq=0, RequestSeq=6", m)
}
// Single current thread is sent when the program is running
// because DAP spec expects at least one thread.
// TODO(polina): accept empty already-terminated response here as well?
if len(m.Body.Threads) != 1 || m.Body.Threads[0].Id != -1 || m.Body.Threads[0].Name != "Current" {
t.Errorf("\ngot %#v\nwant Id=-1, Name=\"Current\"", m.Body.Threads)
}
case *dap.TerminatedEvent:
default:
t.Fatalf("got %#v, want ThreadsResponse or TerminatedEvent", m)
}
}
// 7 >> disconnect, << disconnect
// It is possible for the program to terminate before the initial
// threads request is processed. And in that case, the
// response can be empty
// 7 >> threads, << threads
client.ThreadsRequest()
tResp := client.ExpectThreadsResponse(t)
if tResp.Seq != 0 || tResp.RequestSeq != 7 || len(tResp.Body.Threads) != 0 {
t.Errorf("\ngot %#v\nwant Seq=0, RequestSeq=7 len(Threads)=0", tResp)
}
// 8 >> disconnect, << disconnect
client.DisconnectRequest()
client.ExpectOutputEventProcessExited(t, 0)
client.ExpectOutputEventDetaching(t)
dResp := client.ExpectDisconnectResponse(t)
if dResp.Seq != 0 || dResp.RequestSeq != 7 {
t.Errorf("\ngot %#v\nwant Seq=0, RequestSeq=7", dResp)
if dResp.Seq != 0 || dResp.RequestSeq != 8 {
t.Errorf("\ngot %#v\nwant Seq=0, RequestSeq=8", dResp)
}
})
}
@ -443,19 +484,28 @@ func TestPreSetBreakpoint(t *testing.T) {
client.ConfigurationDoneRequest()
client.ExpectConfigurationDoneResponse(t)
// This triggers "continue"
// This triggers "continue" on a separate goroutine
// TODO(polina): add a no-op threads request
// with dummy response here once server becomes async
// to match what happens in VS Code.
stopEvent1 := client.ExpectStoppedEvent(t)
if stopEvent1.Body.Reason != "breakpoint" ||
stopEvent1.Body.ThreadId != 1 ||
!stopEvent1.Body.AllThreadsStopped {
t.Errorf("got %#v, want Body={Reason=\"breakpoint\", ThreadId=1, AllThreadsStopped=true}", stopEvent1)
client.ThreadsRequest()
// Since we are in async mode while running, we might receive messages in either order.
for i := 0; i < 2; i++ {
msg := client.ExpectMessage(t)
switch m := msg.(type) {
case *dap.ThreadsResponse:
if len(m.Body.Threads) != 1 || m.Body.Threads[0].Id != -1 || m.Body.Threads[0].Name != "Current" {
t.Errorf("\ngot %#v\nwant Id=-1, Name=\"Current\"", m.Body.Threads)
}
case *dap.StoppedEvent:
if m.Body.Reason != "breakpoint" || m.Body.ThreadId != 1 || !m.Body.AllThreadsStopped {
t.Errorf("got %#v, want Body={Reason=\"breakpoint\", ThreadId=1, AllThreadsStopped=true}", m)
}
default:
t.Fatalf("got %#v, want ThreadsResponse or StoppedEvent", m)
}
}
// Threads-StackTrace-Scopes-Variables request waterfall is
// triggered on stop event.
client.ThreadsRequest()
tResp := client.ExpectThreadsResponse(t)
if len(tResp.Body.Threads) < 2 { // 1 main + runtime
@ -2415,14 +2465,14 @@ func TestBadAccess(t *testing.T) {
expectStoppedOnError := func(errorPrefix string) {
t.Helper()
se := client.ExpectStoppedEvent(t)
if se.Body.ThreadId != 1 || se.Body.Reason != "runtime error" || !strings.HasPrefix(se.Body.Text, errorPrefix) {
t.Errorf("\ngot %#v\nwant ThreadId=1 Reason=\"runtime error\" Text=\"%s\"", se, errorPrefix)
}
oe := client.ExpectOutputEvent(t)
if oe.Body.Category != "stderr" || !strings.HasPrefix(oe.Body.Output, "ERROR: "+errorPrefix) {
t.Errorf("\ngot %#v\nwant Category=\"stderr\" Output=\"%s ...\"", oe, errorPrefix)
}
se := client.ExpectStoppedEvent(t)
if se.Body.ThreadId != 1 || se.Body.Reason != "runtime error" || !strings.HasPrefix(se.Body.Text, errorPrefix) {
t.Errorf("\ngot %#v\nwant ThreadId=1 Reason=\"runtime error\" Text=\"%s\"", se, errorPrefix)
}
}
client.ContinueRequest(1)

@ -549,7 +549,7 @@ func (d *Debugger) Restart(rerecord bool, pos string, resetArgs bool, newArgs []
// State returns the current state of the debugger.
func (d *Debugger) State(nowait bool) (*api.DebuggerState, error) {
if d.isRunning() && nowait {
if d.IsRunning() && nowait {
return &api.DebuggerState{Running: true}, nil
}
@ -973,7 +973,7 @@ func (d *Debugger) setRunning(running bool) {
d.runningMutex.Unlock()
}
func (d *Debugger) isRunning() bool {
func (d *Debugger) IsRunning() bool {
d.runningMutex.Lock()
defer d.runningMutex.Unlock()
return d.running
@ -1711,7 +1711,7 @@ func (d *Debugger) GetVersion(out *api.GetVersionOut) error {
}
}
if !d.isRecording() && !d.isRunning() {
if !d.isRecording() && !d.IsRunning() {
out.TargetGoVersion = d.target.BinInfo().Producer()
}