inspector: allow concurrent inspector sessions
This change enables concurrent inspector sessions, through WebSocket interface as well as JS interface, in any combination. PR-URL: https://github.com/nodejs/node/pull/20137 Reviewed-By: James M Snell <jasnell@gmail.com> Reviewed-By: Gus Caplan <me@gus.host> Reviewed-By: Ben Noordhuis <info@bnoordhuis.nl> Reviewed-By: Tiancheng "Timothy" Gu <timothygu99@gmail.com> Reviewed-By: Rich Trott <rtrott@gmail.com>
This commit is contained in:
parent
5c57cea804
commit
1300103534
@ -189,9 +189,9 @@ const int CONTEXT_GROUP_ID = 1;
|
||||
|
||||
class ChannelImpl final : public v8_inspector::V8Inspector::Channel {
|
||||
public:
|
||||
explicit ChannelImpl(V8Inspector* inspector,
|
||||
InspectorSessionDelegate* delegate)
|
||||
: delegate_(delegate) {
|
||||
explicit ChannelImpl(const std::unique_ptr<V8Inspector>& inspector,
|
||||
std::unique_ptr<InspectorSessionDelegate> delegate)
|
||||
: delegate_(std::move(delegate)) {
|
||||
session_ = inspector->connect(1, this, StringView());
|
||||
}
|
||||
|
||||
@ -201,19 +201,11 @@ class ChannelImpl final : public v8_inspector::V8Inspector::Channel {
|
||||
session_->dispatchProtocolMessage(message);
|
||||
}
|
||||
|
||||
bool waitForFrontendMessage() {
|
||||
return delegate_->WaitForFrontendMessageWhilePaused();
|
||||
}
|
||||
|
||||
void schedulePauseOnNextStatement(const std::string& reason) {
|
||||
std::unique_ptr<StringBuffer> buffer = Utf8ToStringView(reason);
|
||||
session_->schedulePauseOnNextStatement(buffer->string(), buffer->string());
|
||||
}
|
||||
|
||||
InspectorSessionDelegate* delegate() {
|
||||
return delegate_;
|
||||
}
|
||||
|
||||
private:
|
||||
void sendResponse(
|
||||
int callId,
|
||||
@ -232,7 +224,7 @@ class ChannelImpl final : public v8_inspector::V8Inspector::Channel {
|
||||
delegate_->SendMessageToFrontend(message);
|
||||
}
|
||||
|
||||
InspectorSessionDelegate* const delegate_;
|
||||
std::unique_ptr<InspectorSessionDelegate> delegate_;
|
||||
std::unique_ptr<v8_inspector::V8InspectorSession> session_;
|
||||
};
|
||||
|
||||
@ -300,8 +292,7 @@ class InspectorTimerHandle {
|
||||
class NodeInspectorClient : public V8InspectorClient {
|
||||
public:
|
||||
NodeInspectorClient(node::Environment* env, node::NodePlatform* platform)
|
||||
: env_(env), platform_(platform), terminated_(false),
|
||||
running_nested_loop_(false) {
|
||||
: env_(env), platform_(platform) {
|
||||
client_ = V8Inspector::create(env->isolate(), this);
|
||||
// TODO(bnoordhuis) Make name configurable from src/node.cc.
|
||||
ContextInfo info(GetHumanReadableProcessName());
|
||||
@ -310,18 +301,28 @@ class NodeInspectorClient : public V8InspectorClient {
|
||||
}
|
||||
|
||||
void runMessageLoopOnPause(int context_group_id) override {
|
||||
CHECK_NE(channel_, nullptr);
|
||||
runMessageLoop(false);
|
||||
}
|
||||
|
||||
void runMessageLoop(bool ignore_terminated) {
|
||||
if (running_nested_loop_)
|
||||
return;
|
||||
terminated_ = false;
|
||||
running_nested_loop_ = true;
|
||||
while (!terminated_ && channel_->waitForFrontendMessage()) {
|
||||
while ((ignore_terminated || !terminated_) && waitForFrontendEvent()) {
|
||||
while (platform_->FlushForegroundTasks(env_->isolate())) {}
|
||||
}
|
||||
terminated_ = false;
|
||||
running_nested_loop_ = false;
|
||||
}
|
||||
|
||||
bool waitForFrontendEvent() {
|
||||
InspectorIo* io = env_->inspector_agent()->io();
|
||||
if (io == nullptr)
|
||||
return false;
|
||||
return io->WaitForFrontendEvent();
|
||||
}
|
||||
|
||||
double currentTimeMS() override {
|
||||
return uv_hrtime() * 1.0 / NANOS_PER_MSEC;
|
||||
}
|
||||
@ -363,20 +364,22 @@ class NodeInspectorClient : public V8InspectorClient {
|
||||
terminated_ = true;
|
||||
}
|
||||
|
||||
void connectFrontend(InspectorSessionDelegate* delegate) {
|
||||
CHECK_EQ(channel_, nullptr);
|
||||
channel_ = std::unique_ptr<ChannelImpl>(
|
||||
new ChannelImpl(client_.get(), delegate));
|
||||
int connectFrontend(std::unique_ptr<InspectorSessionDelegate> delegate) {
|
||||
events_dispatched_ = true;
|
||||
int session_id = next_session_id_++;
|
||||
channels_[session_id] =
|
||||
std::make_unique<ChannelImpl>(client_, std::move(delegate));
|
||||
return session_id;
|
||||
}
|
||||
|
||||
void disconnectFrontend() {
|
||||
quitMessageLoopOnPause();
|
||||
channel_.reset();
|
||||
void disconnectFrontend(int session_id) {
|
||||
events_dispatched_ = true;
|
||||
channels_.erase(session_id);
|
||||
}
|
||||
|
||||
void dispatchMessageFromFrontend(const StringView& message) {
|
||||
CHECK_NE(channel_, nullptr);
|
||||
channel_->dispatchProtocolMessage(message);
|
||||
void dispatchMessageFromFrontend(int session_id, const StringView& message) {
|
||||
events_dispatched_ = true;
|
||||
channels_[session_id]->dispatchProtocolMessage(message);
|
||||
}
|
||||
|
||||
Local<Context> ensureDefaultContextInGroup(int contextGroupId) override {
|
||||
@ -426,10 +429,6 @@ class NodeInspectorClient : public V8InspectorClient {
|
||||
script_id);
|
||||
}
|
||||
|
||||
ChannelImpl* channel() {
|
||||
return channel_.get();
|
||||
}
|
||||
|
||||
void startRepeatingTimer(double interval_s,
|
||||
TimerCallback callback,
|
||||
void* data) override {
|
||||
@ -464,20 +463,31 @@ class NodeInspectorClient : public V8InspectorClient {
|
||||
client_->allAsyncTasksCanceled();
|
||||
}
|
||||
|
||||
void schedulePauseOnNextStatement(const std::string& reason) {
|
||||
for (const auto& id_channel : channels_) {
|
||||
id_channel.second->schedulePauseOnNextStatement(reason);
|
||||
}
|
||||
}
|
||||
|
||||
bool hasConnectedSessions() {
|
||||
return !channels_.empty();
|
||||
}
|
||||
|
||||
private:
|
||||
node::Environment* env_;
|
||||
node::NodePlatform* platform_;
|
||||
bool terminated_;
|
||||
bool running_nested_loop_;
|
||||
bool terminated_ = false;
|
||||
bool running_nested_loop_ = false;
|
||||
std::unique_ptr<V8Inspector> client_;
|
||||
std::unique_ptr<ChannelImpl> channel_;
|
||||
std::unordered_map<int, std::unique_ptr<ChannelImpl>> channels_;
|
||||
std::unordered_map<void*, InspectorTimerHandle> timers_;
|
||||
int next_session_id_ = 1;
|
||||
bool events_dispatched_ = false;
|
||||
};
|
||||
|
||||
Agent::Agent(Environment* env) : parent_env_(env),
|
||||
client_(nullptr),
|
||||
platform_(nullptr),
|
||||
enabled_(false),
|
||||
pending_enable_async_hook_(false),
|
||||
pending_disable_async_hook_(false) {}
|
||||
|
||||
@ -491,7 +501,7 @@ bool Agent::Start(node::NodePlatform* platform, const char* path,
|
||||
path_ = path == nullptr ? "" : path;
|
||||
debug_options_ = options;
|
||||
client_ =
|
||||
std::unique_ptr<NodeInspectorClient>(
|
||||
std::shared_ptr<NodeInspectorClient>(
|
||||
new NodeInspectorClient(parent_env_, platform));
|
||||
platform_ = platform;
|
||||
CHECK_EQ(0, uv_async_init(uv_default_loop(),
|
||||
@ -515,7 +525,6 @@ bool Agent::StartIoThread(bool wait_for_connect) {
|
||||
|
||||
CHECK_NE(client_, nullptr);
|
||||
|
||||
enabled_ = true;
|
||||
io_ = std::unique_ptr<InspectorIo>(
|
||||
new InspectorIo(parent_env_, platform_, path_, debug_options_,
|
||||
wait_for_connect));
|
||||
@ -554,13 +563,13 @@ void Agent::Stop() {
|
||||
if (io_ != nullptr) {
|
||||
io_->Stop();
|
||||
io_.reset();
|
||||
enabled_ = false;
|
||||
}
|
||||
}
|
||||
|
||||
void Agent::Connect(InspectorSessionDelegate* delegate) {
|
||||
enabled_ = true;
|
||||
client_->connectFrontend(delegate);
|
||||
std::unique_ptr<InspectorSession> Agent::Connect(
|
||||
std::unique_ptr<InspectorSessionDelegate> delegate) {
|
||||
int session_id = client_->connectFrontend(std::move(delegate));
|
||||
return std::make_unique<InspectorSession>(session_id, client_);
|
||||
}
|
||||
|
||||
void Agent::WaitForDisconnect() {
|
||||
@ -568,6 +577,11 @@ void Agent::WaitForDisconnect() {
|
||||
client_->contextDestroyed(parent_env_->context());
|
||||
if (io_ != nullptr) {
|
||||
io_->WaitForDisconnect();
|
||||
// There is a bug in V8 Inspector (https://crbug.com/834056) that
|
||||
// calls V8InspectorClient::quitMessageLoopOnPause when a session
|
||||
// disconnects. We are using this flag to ignore those calls so the message
|
||||
// loop is spinning as long as there's a reason to expect inspector messages
|
||||
client_->runMessageLoop(true);
|
||||
}
|
||||
}
|
||||
|
||||
@ -578,33 +592,8 @@ void Agent::FatalException(Local<Value> error, Local<v8::Message> message) {
|
||||
WaitForDisconnect();
|
||||
}
|
||||
|
||||
void Agent::Dispatch(const StringView& message) {
|
||||
CHECK_NE(client_, nullptr);
|
||||
client_->dispatchMessageFromFrontend(message);
|
||||
}
|
||||
|
||||
void Agent::Disconnect() {
|
||||
CHECK_NE(client_, nullptr);
|
||||
client_->disconnectFrontend();
|
||||
}
|
||||
|
||||
void Agent::RunMessageLoop() {
|
||||
CHECK_NE(client_, nullptr);
|
||||
client_->runMessageLoopOnPause(CONTEXT_GROUP_ID);
|
||||
}
|
||||
|
||||
InspectorSessionDelegate* Agent::delegate() {
|
||||
CHECK_NE(client_, nullptr);
|
||||
ChannelImpl* channel = client_->channel();
|
||||
if (channel == nullptr)
|
||||
return nullptr;
|
||||
return channel->delegate();
|
||||
}
|
||||
|
||||
void Agent::PauseOnNextJavascriptStatement(const std::string& reason) {
|
||||
ChannelImpl* channel = client_->channel();
|
||||
if (channel != nullptr)
|
||||
channel->schedulePauseOnNextStatement(reason);
|
||||
client_->schedulePauseOnNextStatement(reason);
|
||||
}
|
||||
|
||||
void Agent::RegisterAsyncHook(Isolate* isolate,
|
||||
@ -699,5 +688,20 @@ bool Agent::IsWaitingForConnect() {
|
||||
return debug_options_.wait_for_connect();
|
||||
}
|
||||
|
||||
bool Agent::HasConnectedSessions() {
|
||||
return client_->hasConnectedSessions();
|
||||
}
|
||||
|
||||
InspectorSession::InspectorSession(int session_id,
|
||||
std::shared_ptr<NodeInspectorClient> client)
|
||||
: session_id_(session_id), client_(client) {}
|
||||
|
||||
InspectorSession::~InspectorSession() {
|
||||
client_->disconnectFrontend(session_id_);
|
||||
}
|
||||
|
||||
void InspectorSession::Dispatch(const StringView& message) {
|
||||
client_->dispatchMessageFromFrontend(session_id_, message);
|
||||
}
|
||||
} // namespace inspector
|
||||
} // namespace node
|
||||
|
@ -23,18 +23,26 @@ class Environment;
|
||||
struct ContextInfo;
|
||||
|
||||
namespace inspector {
|
||||
class InspectorIo;
|
||||
class NodeInspectorClient;
|
||||
|
||||
class InspectorSession {
|
||||
public:
|
||||
InspectorSession(int session_id, std::shared_ptr<NodeInspectorClient> client);
|
||||
~InspectorSession();
|
||||
void Dispatch(const v8_inspector::StringView& message);
|
||||
private:
|
||||
int session_id_;
|
||||
std::shared_ptr<NodeInspectorClient> client_;
|
||||
};
|
||||
|
||||
class InspectorSessionDelegate {
|
||||
public:
|
||||
virtual ~InspectorSessionDelegate() = default;
|
||||
virtual bool WaitForFrontendMessageWhilePaused() = 0;
|
||||
virtual void SendMessageToFrontend(const v8_inspector::StringView& message)
|
||||
= 0;
|
||||
};
|
||||
|
||||
class InspectorIo;
|
||||
class NodeInspectorClient;
|
||||
|
||||
class Agent {
|
||||
public:
|
||||
explicit Agent(node::Environment* env);
|
||||
@ -66,19 +74,19 @@ class Agent {
|
||||
void RegisterAsyncHook(v8::Isolate* isolate,
|
||||
v8::Local<v8::Function> enable_function,
|
||||
v8::Local<v8::Function> disable_function);
|
||||
void EnableAsyncHook();
|
||||
void DisableAsyncHook();
|
||||
|
||||
// These methods are called by the WS protocol and JS binding to create
|
||||
// inspector sessions. The inspector responds by using the delegate to send
|
||||
// messages back.
|
||||
void Connect(InspectorSessionDelegate* delegate);
|
||||
void Disconnect();
|
||||
void Dispatch(const v8_inspector::StringView& message);
|
||||
InspectorSessionDelegate* delegate();
|
||||
// Called by the WS protocol and JS binding to create inspector sessions.
|
||||
// The inspector responds by using the delegate to send messages back.
|
||||
std::unique_ptr<InspectorSession> Connect(
|
||||
std::unique_ptr<InspectorSessionDelegate> delegate);
|
||||
|
||||
void RunMessageLoop();
|
||||
bool enabled() { return enabled_; }
|
||||
void PauseOnNextJavascriptStatement(const std::string& reason);
|
||||
|
||||
// Returns true as long as there is at least one connected session.
|
||||
bool HasConnectedSessions();
|
||||
|
||||
InspectorIo* io() {
|
||||
return io_.get();
|
||||
}
|
||||
@ -92,18 +100,14 @@ class Agent {
|
||||
DebugOptions& options() { return debug_options_; }
|
||||
void ContextCreated(v8::Local<v8::Context> context, const ContextInfo& info);
|
||||
|
||||
void EnableAsyncHook();
|
||||
void DisableAsyncHook();
|
||||
|
||||
private:
|
||||
void ToggleAsyncHook(v8::Isolate* isolate,
|
||||
const Persistent<v8::Function>& fn);
|
||||
|
||||
node::Environment* parent_env_;
|
||||
std::unique_ptr<NodeInspectorClient> client_;
|
||||
std::shared_ptr<NodeInspectorClient> client_;
|
||||
std::unique_ptr<InspectorIo> io_;
|
||||
v8::Platform* platform_;
|
||||
bool enabled_;
|
||||
std::string path_;
|
||||
DebugOptions debug_options_;
|
||||
|
||||
|
@ -122,11 +122,11 @@ std::unique_ptr<StringBuffer> Utf8ToStringView(const std::string& message) {
|
||||
|
||||
class IoSessionDelegate : public InspectorSessionDelegate {
|
||||
public:
|
||||
explicit IoSessionDelegate(InspectorIo* io) : io_(io) { }
|
||||
bool WaitForFrontendMessageWhilePaused() override;
|
||||
explicit IoSessionDelegate(InspectorIo* io, int id) : io_(io), id_(id) { }
|
||||
void SendMessageToFrontend(const v8_inspector::StringView& message) override;
|
||||
private:
|
||||
InspectorIo* io_;
|
||||
int id_;
|
||||
};
|
||||
|
||||
// Passed to InspectorSocketServer to handle WS inspector protocol events,
|
||||
@ -190,8 +190,7 @@ InspectorIo::InspectorIo(Environment* env, v8::Platform* platform,
|
||||
: options_(options), thread_(), delegate_(nullptr),
|
||||
state_(State::kNew), parent_env_(env),
|
||||
thread_req_(), platform_(platform),
|
||||
dispatching_messages_(false), session_id_(0),
|
||||
script_name_(path),
|
||||
dispatching_messages_(false), script_name_(path),
|
||||
wait_for_connect_(wait_for_connect), port_(-1) {
|
||||
main_thread_req_ = new AsyncAndAgent({uv_async_t(), env->inspector_agent()});
|
||||
CHECK_EQ(0, uv_async_init(env->event_loop(), &main_thread_req_->first,
|
||||
@ -222,7 +221,7 @@ bool InspectorIo::Start() {
|
||||
}
|
||||
|
||||
void InspectorIo::Stop() {
|
||||
CHECK(state_ == State::kAccepting || state_ == State::kConnected);
|
||||
CHECK(state_ == State::kAccepting || !sessions_.empty());
|
||||
Write(TransportAction::kKill, 0, StringView());
|
||||
int err = uv_thread_join(&thread_);
|
||||
CHECK_EQ(err, 0);
|
||||
@ -237,12 +236,11 @@ bool InspectorIo::IsStarted() {
|
||||
void InspectorIo::WaitForDisconnect() {
|
||||
if (state_ == State::kAccepting)
|
||||
state_ = State::kDone;
|
||||
if (state_ == State::kConnected) {
|
||||
if (!sessions_.empty()) {
|
||||
state_ = State::kShutDown;
|
||||
Write(TransportAction::kStop, 0, StringView());
|
||||
fprintf(stderr, "Waiting for the debugger to disconnect...\n");
|
||||
fflush(stderr);
|
||||
parent_env_->inspector_agent()->RunMessageLoop();
|
||||
}
|
||||
}
|
||||
|
||||
@ -348,45 +346,23 @@ void InspectorIo::PostIncomingMessage(InspectorAction action, int session_id,
|
||||
isolate->RequestInterrupt(InterruptCallback, agent);
|
||||
CHECK_EQ(0, uv_async_send(&main_thread_req_->first));
|
||||
}
|
||||
NotifyMessageReceived();
|
||||
Mutex::ScopedLock scoped_lock(state_lock_);
|
||||
incoming_message_cond_.Broadcast(scoped_lock);
|
||||
}
|
||||
|
||||
std::vector<std::string> InspectorIo::GetTargetIds() const {
|
||||
return delegate_ ? delegate_->GetTargetIds() : std::vector<std::string>();
|
||||
}
|
||||
|
||||
void InspectorIo::WaitForFrontendMessageWhilePaused() {
|
||||
dispatching_messages_ = false;
|
||||
Mutex::ScopedLock scoped_lock(state_lock_);
|
||||
if (incoming_message_queue_.empty())
|
||||
incoming_message_cond_.Wait(scoped_lock);
|
||||
}
|
||||
|
||||
void InspectorIo::NotifyMessageReceived() {
|
||||
Mutex::ScopedLock scoped_lock(state_lock_);
|
||||
incoming_message_cond_.Broadcast(scoped_lock);
|
||||
}
|
||||
|
||||
TransportAction InspectorIo::Attach(int session_id) {
|
||||
Agent* agent = parent_env_->inspector_agent();
|
||||
if (agent->delegate() != nullptr)
|
||||
return TransportAction::kDeclineSession;
|
||||
|
||||
CHECK_EQ(session_delegate_, nullptr);
|
||||
session_id_ = session_id;
|
||||
state_ = State::kConnected;
|
||||
fprintf(stderr, "Debugger attached.\n");
|
||||
session_delegate_ = std::unique_ptr<InspectorSessionDelegate>(
|
||||
new IoSessionDelegate(this));
|
||||
agent->Connect(session_delegate_.get());
|
||||
sessions_[session_id] =
|
||||
agent->Connect(std::make_unique<IoSessionDelegate>(this, session_id));
|
||||
return TransportAction::kAcceptSession;
|
||||
}
|
||||
|
||||
void InspectorIo::DispatchMessages() {
|
||||
// This function can be reentered if there was an incoming message while
|
||||
// V8 was processing another inspector request (e.g. if the user is
|
||||
// evaluating a long-running JS code snippet). This can happen only at
|
||||
// specific points (e.g. the lines that call inspector_ methods)
|
||||
if (dispatching_messages_)
|
||||
return;
|
||||
dispatching_messages_ = true;
|
||||
@ -409,17 +385,20 @@ void InspectorIo::DispatchMessages() {
|
||||
Attach(id);
|
||||
break;
|
||||
case InspectorAction::kEndSession:
|
||||
CHECK_NE(session_delegate_, nullptr);
|
||||
sessions_.erase(id);
|
||||
if (!sessions_.empty())
|
||||
continue;
|
||||
if (state_ == State::kShutDown) {
|
||||
state_ = State::kDone;
|
||||
} else {
|
||||
state_ = State::kAccepting;
|
||||
}
|
||||
parent_env_->inspector_agent()->Disconnect();
|
||||
session_delegate_.reset();
|
||||
break;
|
||||
case InspectorAction::kSendMessage:
|
||||
parent_env_->inspector_agent()->Dispatch(message);
|
||||
auto session = sessions_.find(id);
|
||||
if (session != sessions_.end() && session->second) {
|
||||
session->second->Dispatch(message);
|
||||
}
|
||||
break;
|
||||
}
|
||||
}
|
||||
@ -445,6 +424,20 @@ void InspectorIo::Write(TransportAction action, int session_id,
|
||||
CHECK_EQ(0, err);
|
||||
}
|
||||
|
||||
bool InspectorIo::WaitForFrontendEvent() {
|
||||
// We allow DispatchMessages reentry as we enter the pause. This is important
|
||||
// to support debugging the code invoked by an inspector call, such
|
||||
// as Runtime.evaluate
|
||||
dispatching_messages_ = false;
|
||||
Mutex::ScopedLock scoped_lock(state_lock_);
|
||||
if (sessions_.empty())
|
||||
return false;
|
||||
if (dispatching_message_queue_.empty() && incoming_message_queue_.empty()) {
|
||||
incoming_message_cond_.Wait(scoped_lock);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
InspectorIoDelegate::InspectorIoDelegate(InspectorIo* io,
|
||||
const std::string& script_path,
|
||||
const std::string& script_name,
|
||||
@ -502,14 +495,9 @@ std::string InspectorIoDelegate::GetTargetUrl(const std::string& id) {
|
||||
return "file://" + script_path_;
|
||||
}
|
||||
|
||||
bool IoSessionDelegate::WaitForFrontendMessageWhilePaused() {
|
||||
io_->WaitForFrontendMessageWhilePaused();
|
||||
return true;
|
||||
}
|
||||
|
||||
void IoSessionDelegate::SendMessageToFrontend(
|
||||
const v8_inspector::StringView& message) {
|
||||
io_->Write(TransportAction::kSendMessage, io_->session_id_, message);
|
||||
io_->Write(TransportAction::kSendMessage, id_, message);
|
||||
}
|
||||
|
||||
} // namespace inspector
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include "uv.h"
|
||||
|
||||
#include <deque>
|
||||
#include <map>
|
||||
#include <memory>
|
||||
#include <stddef.h>
|
||||
|
||||
@ -76,6 +77,7 @@ class InspectorIo {
|
||||
void ServerDone() {
|
||||
uv_close(reinterpret_cast<uv_handle_t*>(&thread_req_), nullptr);
|
||||
}
|
||||
bool WaitForFrontendEvent();
|
||||
|
||||
int port() const { return port_; }
|
||||
std::string host() const { return options_.host_name(); }
|
||||
@ -89,7 +91,6 @@ class InspectorIo {
|
||||
enum class State {
|
||||
kNew,
|
||||
kAccepting,
|
||||
kConnected,
|
||||
kDone,
|
||||
kError,
|
||||
kShutDown
|
||||
@ -107,7 +108,6 @@ class InspectorIo {
|
||||
// messages from outgoing_message_queue to the InspectorSockerServer
|
||||
template <typename Transport> static void IoThreadAsyncCb(uv_async_t* async);
|
||||
|
||||
void SetConnected(bool connected);
|
||||
void DispatchMessages();
|
||||
// Write action to outgoing_message_queue, and wake the thread
|
||||
void Write(TransportAction action, int session_id,
|
||||
@ -122,10 +122,6 @@ class InspectorIo {
|
||||
template <typename ActionType>
|
||||
void SwapBehindLock(MessageQueue<ActionType>* vector1,
|
||||
MessageQueue<ActionType>* vector2);
|
||||
// Wait on incoming_message_cond_
|
||||
void WaitForFrontendMessageWhilePaused();
|
||||
// Broadcast incoming_message_cond_
|
||||
void NotifyMessageReceived();
|
||||
// Attach session to an inspector. Either kAcceptSession or kDeclineSession
|
||||
TransportAction Attach(int session_id);
|
||||
|
||||
@ -147,7 +143,6 @@ class InspectorIo {
|
||||
// Note that this will live while the async is being closed - likely, past
|
||||
// the parent object lifespan
|
||||
std::pair<uv_async_t, Agent*>* main_thread_req_;
|
||||
std::unique_ptr<InspectorSessionDelegate> session_delegate_;
|
||||
v8::Platform* platform_;
|
||||
|
||||
// Message queues
|
||||
@ -155,15 +150,17 @@ class InspectorIo {
|
||||
Mutex state_lock_; // Locked before mutating either queue.
|
||||
MessageQueue<InspectorAction> incoming_message_queue_;
|
||||
MessageQueue<TransportAction> outgoing_message_queue_;
|
||||
// This queue is to maintain the order of the messages for the cases
|
||||
// when we reenter the DispatchMessages function.
|
||||
MessageQueue<InspectorAction> dispatching_message_queue_;
|
||||
|
||||
bool dispatching_messages_;
|
||||
int session_id_;
|
||||
|
||||
std::string script_name_;
|
||||
std::string script_path_;
|
||||
const bool wait_for_connect_;
|
||||
int port_;
|
||||
std::unordered_map<int, std::unique_ptr<InspectorSession>> sessions_;
|
||||
|
||||
friend class DispatchMessagesTask;
|
||||
friend class IoSessionDelegate;
|
||||
|
@ -42,10 +42,6 @@ class JSBindingsConnection : public AsyncWrap {
|
||||
connection_(connection) {
|
||||
}
|
||||
|
||||
bool WaitForFrontendMessageWhilePaused() override {
|
||||
return false;
|
||||
}
|
||||
|
||||
void SendMessageToFrontend(const v8_inspector::StringView& message)
|
||||
override {
|
||||
Isolate* isolate = env_->isolate();
|
||||
@ -58,12 +54,6 @@ class JSBindingsConnection : public AsyncWrap {
|
||||
connection_->OnMessage(argument);
|
||||
}
|
||||
|
||||
void Disconnect() {
|
||||
Agent* agent = env_->inspector_agent();
|
||||
if (agent->delegate() == this)
|
||||
agent->Disconnect();
|
||||
}
|
||||
|
||||
private:
|
||||
Environment* env_;
|
||||
JSBindingsConnection* connection_;
|
||||
@ -73,31 +63,17 @@ class JSBindingsConnection : public AsyncWrap {
|
||||
Local<Object> wrap,
|
||||
Local<Function> callback)
|
||||
: AsyncWrap(env, wrap, PROVIDER_INSPECTORJSBINDING),
|
||||
delegate_(env, this),
|
||||
callback_(env->isolate(), callback) {
|
||||
Wrap(wrap, this);
|
||||
|
||||
Agent* inspector = env->inspector_agent();
|
||||
if (inspector->delegate() != nullptr) {
|
||||
// This signals JS code that it has to throw an error.
|
||||
Local<String> session_attached =
|
||||
FIXED_ONE_BYTE_STRING(env->isolate(), "sessionAttached");
|
||||
wrap->Set(env->context(), session_attached,
|
||||
Boolean::New(env->isolate(), true)).ToChecked();
|
||||
return;
|
||||
}
|
||||
inspector->Connect(&delegate_);
|
||||
session_ = inspector->Connect(
|
||||
std::make_unique<JSBindingsSessionDelegate>(env, this));
|
||||
}
|
||||
|
||||
void OnMessage(Local<Value> value) {
|
||||
MakeCallback(callback_.Get(env()->isolate()), 1, &value);
|
||||
}
|
||||
|
||||
void CheckIsCurrent() {
|
||||
Agent* inspector = env()->inspector_agent();
|
||||
CHECK_EQ(&delegate_, inspector->delegate());
|
||||
}
|
||||
|
||||
static void New(const FunctionCallbackInfo<Value>& info) {
|
||||
Environment* env = Environment::GetCurrent(info);
|
||||
CHECK(info[0]->IsFunction());
|
||||
@ -106,7 +82,7 @@ class JSBindingsConnection : public AsyncWrap {
|
||||
}
|
||||
|
||||
void Disconnect() {
|
||||
delegate_.Disconnect();
|
||||
session_.reset();
|
||||
if (!persistent().IsEmpty()) {
|
||||
ClearWrap(object());
|
||||
}
|
||||
@ -125,18 +101,23 @@ class JSBindingsConnection : public AsyncWrap {
|
||||
ASSIGN_OR_RETURN_UNWRAP(&session, info.Holder());
|
||||
CHECK(info[0]->IsString());
|
||||
|
||||
session->CheckIsCurrent();
|
||||
Agent* inspector = env->inspector_agent();
|
||||
inspector->Dispatch(ToProtocolString(env->isolate(), info[0])->string());
|
||||
if (session->session_) {
|
||||
session->session_->Dispatch(
|
||||
ToProtocolString(env->isolate(), info[0])->string());
|
||||
}
|
||||
}
|
||||
|
||||
size_t self_size() const override { return sizeof(*this); }
|
||||
|
||||
private:
|
||||
JSBindingsSessionDelegate delegate_;
|
||||
std::unique_ptr<InspectorSession> session_;
|
||||
Persistent<Function> callback_;
|
||||
};
|
||||
|
||||
static bool InspectorEnabled(Environment* env) {
|
||||
Agent* agent = env->inspector_agent();
|
||||
return agent->io() != nullptr || agent->HasConnectedSessions();
|
||||
}
|
||||
|
||||
void AddCommandLineAPI(const FunctionCallbackInfo<Value>& info) {
|
||||
auto env = Environment::GetCurrent(info);
|
||||
@ -178,7 +159,7 @@ void InspectorConsoleCall(const FunctionCallbackInfo<Value>& info) {
|
||||
call_args.push_back(info[i]);
|
||||
}
|
||||
Environment* env = Environment::GetCurrent(isolate);
|
||||
if (env->inspector_agent()->enabled()) {
|
||||
if (InspectorEnabled(env)) {
|
||||
Local<Value> inspector_method = info[0];
|
||||
CHECK(inspector_method->IsFunction());
|
||||
Local<Value> config_value = info[2];
|
||||
@ -256,7 +237,7 @@ static void RegisterAsyncHookWrapper(const FunctionCallbackInfo<Value>& args) {
|
||||
|
||||
void IsEnabled(const FunctionCallbackInfo<Value>& args) {
|
||||
Environment* env = Environment::GetCurrent(args);
|
||||
args.GetReturnValue().Set(env->inspector_agent()->enabled());
|
||||
args.GetReturnValue().Set(InspectorEnabled(env));
|
||||
}
|
||||
|
||||
void Open(const FunctionCallbackInfo<Value>& args) {
|
||||
|
@ -373,27 +373,17 @@ void InspectorSocketServer::SendListResponse(InspectorSocket* socket,
|
||||
target_map["url"] = delegate_->GetTargetUrl(id);
|
||||
Escape(&target_map["url"]);
|
||||
|
||||
bool connected = false;
|
||||
for (const auto& session : connected_sessions_) {
|
||||
if (session.second.first == id) {
|
||||
connected = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (!connected) {
|
||||
std::string detected_host = host;
|
||||
if (detected_host.empty()) {
|
||||
detected_host = FormatHostPort(socket->GetHost(),
|
||||
session->server_port());
|
||||
}
|
||||
std::ostringstream frontend_url;
|
||||
frontend_url << "chrome-devtools://devtools/bundled";
|
||||
frontend_url << "/inspector.html?experiments=true&v8only=true&ws=";
|
||||
frontend_url << FormatAddress(detected_host, id, false);
|
||||
target_map["devtoolsFrontendUrl"] += frontend_url.str();
|
||||
target_map["webSocketDebuggerUrl"] =
|
||||
FormatAddress(detected_host, id, true);
|
||||
std::string detected_host = host;
|
||||
if (detected_host.empty()) {
|
||||
detected_host = FormatHostPort(socket->GetHost(),
|
||||
session->server_port());
|
||||
}
|
||||
std::ostringstream frontend_url;
|
||||
frontend_url << "chrome-devtools://devtools/bundled";
|
||||
frontend_url << "/inspector.html?experiments=true&v8only=true&ws=";
|
||||
frontend_url << FormatAddress(detected_host, id, false);
|
||||
target_map["devtoolsFrontendUrl"] += frontend_url.str();
|
||||
target_map["webSocketDebuggerUrl"] = FormatAddress(detected_host, id, true);
|
||||
}
|
||||
SendHttpResponse(socket, MapsToString(response));
|
||||
}
|
||||
|
@ -1959,7 +1959,7 @@ static void InitGroups(const FunctionCallbackInfo<Value>& args) {
|
||||
|
||||
static void WaitForInspectorDisconnect(Environment* env) {
|
||||
#if HAVE_INSPECTOR
|
||||
if (env->inspector_agent()->delegate() != nullptr) {
|
||||
if (env->inspector_agent()->HasConnectedSessions()) {
|
||||
// Restore signal dispositions, the app is done and is no longer
|
||||
// capable of handling signals.
|
||||
#if defined(__POSIX__) && !defined(NODE_SHARED_MODE)
|
||||
|
@ -154,8 +154,9 @@ class InspectorSession {
|
||||
return this._terminationPromise;
|
||||
}
|
||||
|
||||
disconnect() {
|
||||
async disconnect() {
|
||||
this._socket.destroy();
|
||||
return this.waitForServerDisconnect();
|
||||
}
|
||||
|
||||
_onMessage(message) {
|
||||
@ -444,6 +445,7 @@ class NodeInstance {
|
||||
|
||||
kill() {
|
||||
this._process.kill();
|
||||
return this.expectShutdown();
|
||||
}
|
||||
|
||||
scriptPath() {
|
||||
|
@ -9,12 +9,6 @@ const { resolve: UrlResolve } = require('url');
|
||||
const fixtures = require('../common/fixtures');
|
||||
const { NodeInstance } = require('../common/inspector-helper.js');
|
||||
|
||||
function assertNoUrlsWhileConnected(response) {
|
||||
assert.strictEqual(response.length, 1);
|
||||
assert.ok(!response[0].hasOwnProperty('devtoolsFrontendUrl'));
|
||||
assert.ok(!response[0].hasOwnProperty('webSocketDebuggerUrl'));
|
||||
}
|
||||
|
||||
function assertScopeValues({ result }, expected) {
|
||||
const unmatched = new Set(Object.keys(expected));
|
||||
for (const actual of result) {
|
||||
@ -110,7 +104,6 @@ async function runTest() {
|
||||
'', fixtures.path('es-modules/loop.mjs'));
|
||||
|
||||
const session = await child.connectInspectorSession();
|
||||
assertNoUrlsWhileConnected(await child.httpGet(null, '/json/list'));
|
||||
await testBreakpointOnStart(session);
|
||||
await testBreakpoint(session);
|
||||
await session.runToCompletion();
|
||||
|
59
test/parallel/test-inspector-multisession-js.js
Normal file
59
test/parallel/test-inspector-multisession-js.js
Normal file
@ -0,0 +1,59 @@
|
||||
'use strict';
|
||||
const common = require('../common');
|
||||
|
||||
common.skipIfInspectorDisabled();
|
||||
|
||||
const assert = require('assert');
|
||||
const { Session } = require('inspector');
|
||||
const path = require('path');
|
||||
|
||||
function debugged() {
|
||||
return 42;
|
||||
}
|
||||
|
||||
async function test() {
|
||||
const session1 = new Session();
|
||||
const session2 = new Session();
|
||||
|
||||
session1.connect();
|
||||
session2.connect();
|
||||
|
||||
let session1Paused = false;
|
||||
let session2Paused = false;
|
||||
|
||||
session1.on('Debugger.paused', () => session1Paused = true);
|
||||
session2.on('Debugger.paused', () => session2Paused = true);
|
||||
|
||||
console.log('Connected');
|
||||
|
||||
session1.post('Debugger.enable');
|
||||
session2.post('Debugger.enable');
|
||||
console.log('Debugger was enabled');
|
||||
|
||||
await new Promise((resolve, reject) => {
|
||||
session1.post('Debugger.setBreakpointByUrl', {
|
||||
'lineNumber': 9,
|
||||
'url': path.resolve(__dirname, __filename),
|
||||
'columnNumber': 0,
|
||||
'condition': ''
|
||||
}, (error, result) => {
|
||||
return error ? reject(error) : resolve(result);
|
||||
});
|
||||
});
|
||||
console.log('Breakpoint was set');
|
||||
|
||||
debugged();
|
||||
|
||||
// Both sessions will receive the paused event
|
||||
assert(session1Paused);
|
||||
assert(session2Paused);
|
||||
console.log('Breakpoint was hit');
|
||||
|
||||
session1.disconnect();
|
||||
session2.disconnect();
|
||||
console.log('Sessions were disconnected');
|
||||
}
|
||||
|
||||
common.crashOnUnhandledRejection();
|
||||
|
||||
test();
|
75
test/parallel/test-inspector-multisession-ws.js
Normal file
75
test/parallel/test-inspector-multisession-ws.js
Normal file
@ -0,0 +1,75 @@
|
||||
// Flags: --expose-internals
|
||||
'use strict';
|
||||
const common = require('../common');
|
||||
|
||||
common.skipIfInspectorDisabled();
|
||||
|
||||
const { NodeInstance } = require('../common/inspector-helper.js');
|
||||
|
||||
// Sets up JS bindings session and runs till the "paused" event
|
||||
const script = `
|
||||
const { Session } = require('inspector');
|
||||
const session = new Session();
|
||||
let done = false;
|
||||
const interval = setInterval(() => {
|
||||
if (done)
|
||||
clearInterval(interval);
|
||||
}, 150);
|
||||
session.on('Debugger.paused', () => {
|
||||
done = true;
|
||||
});
|
||||
session.connect();
|
||||
session.post('Debugger.enable');
|
||||
console.log('Ready');
|
||||
`;
|
||||
|
||||
async function setupSession(node) {
|
||||
const session = await node.connectInspectorSession();
|
||||
await session.send([
|
||||
{ 'method': 'Runtime.enable' },
|
||||
{ 'method': 'Debugger.enable' },
|
||||
{ 'method': 'Debugger.setPauseOnExceptions',
|
||||
'params': { 'state': 'none' } },
|
||||
{ 'method': 'Debugger.setAsyncCallStackDepth',
|
||||
'params': { 'maxDepth': 0 } },
|
||||
{ 'method': 'Profiler.enable' },
|
||||
{ 'method': 'Profiler.setSamplingInterval',
|
||||
'params': { 'interval': 100 } },
|
||||
{ 'method': 'Debugger.setBlackboxPatterns',
|
||||
'params': { 'patterns': [] } },
|
||||
{ 'method': 'Runtime.runIfWaitingForDebugger' }
|
||||
]);
|
||||
return session;
|
||||
}
|
||||
|
||||
async function testSuspend(sessionA, sessionB) {
|
||||
console.log('[test]', 'Breaking in code and verifying events are fired');
|
||||
await sessionA.waitForNotification('Debugger.paused', 'Initial pause');
|
||||
sessionA.send({ 'method': 'Debugger.resume' });
|
||||
|
||||
await sessionA.waitForNotification('Runtime.consoleAPICalled',
|
||||
'Console output');
|
||||
sessionA.send({ 'method': 'Debugger.pause' });
|
||||
return Promise.all([
|
||||
sessionA.waitForNotification('Debugger.paused', 'SessionA paused'),
|
||||
sessionB.waitForNotification('Debugger.paused', 'SessionB paused')
|
||||
]);
|
||||
}
|
||||
|
||||
async function runTest() {
|
||||
const child = new NodeInstance(undefined, script);
|
||||
|
||||
const [session1, session2] =
|
||||
await Promise.all([setupSession(child), setupSession(child)]);
|
||||
await testSuspend(session2, session1);
|
||||
console.log('[test]', 'Should shut down after both sessions disconnect');
|
||||
|
||||
await session1.runToCompletion();
|
||||
await session2.send({ 'method': 'Debugger.disable' });
|
||||
await session2.disconnect();
|
||||
return child.expectShutdown();
|
||||
}
|
||||
|
||||
common.crashOnUnhandledRejection();
|
||||
|
||||
runTest();
|
@ -1,31 +0,0 @@
|
||||
// Flags: --expose-internals
|
||||
'use strict';
|
||||
const common = require('../common');
|
||||
common.skipIfInspectorDisabled();
|
||||
common.crashOnUnhandledRejection();
|
||||
const { NodeInstance } = require('../common/inspector-helper.js');
|
||||
const assert = require('assert');
|
||||
|
||||
const expected = 'Can connect now!';
|
||||
|
||||
const script = `
|
||||
'use strict';
|
||||
const { Session } = require('inspector');
|
||||
|
||||
const s = new Session();
|
||||
s.connect();
|
||||
console.error('${expected}');
|
||||
process.stdin.on('data', () => process.exit(0));
|
||||
`;
|
||||
|
||||
async function runTests() {
|
||||
const instance = new NodeInstance(['--inspect=0', '--expose-internals'],
|
||||
script);
|
||||
while (await instance.nextStderrString() !== expected);
|
||||
assert.strictEqual(400, await instance.expectConnectionDeclined());
|
||||
instance.write('Stop!\n');
|
||||
assert.deepStrictEqual({ exitCode: 0, signal: null },
|
||||
await instance.expectShutdown());
|
||||
}
|
||||
|
||||
runTests();
|
@ -76,15 +76,6 @@ function testSampleDebugSession() {
|
||||
};
|
||||
const session = new inspector.Session();
|
||||
session.connect();
|
||||
let secondSessionOpened = false;
|
||||
const secondSession = new inspector.Session();
|
||||
try {
|
||||
secondSession.connect();
|
||||
secondSessionOpened = true;
|
||||
} catch (error) {
|
||||
// expected as the session already exists
|
||||
}
|
||||
assert.strictEqual(secondSessionOpened, false);
|
||||
session.on('Debugger.paused',
|
||||
(notification) => debuggerPausedCallback(session, notification));
|
||||
let cbAsSecondArgCalled = false;
|
||||
|
@ -55,16 +55,6 @@ common.expectsError(
|
||||
}
|
||||
);
|
||||
|
||||
const session2 = new Session();
|
||||
common.expectsError(
|
||||
() => session2.connect(),
|
||||
{
|
||||
code: 'ERR_INSPECTOR_ALREADY_CONNECTED',
|
||||
type: Error,
|
||||
message: 'Another inspector session is already connected'
|
||||
}
|
||||
);
|
||||
|
||||
session.disconnect();
|
||||
// Calling disconnect twice should not throw.
|
||||
session.disconnect();
|
||||
|
@ -37,12 +37,6 @@ function checkException(message) {
|
||||
'An exception occurred during execution');
|
||||
}
|
||||
|
||||
function assertNoUrlsWhileConnected(response) {
|
||||
assert.strictEqual(1, response.length);
|
||||
assert.ok(!response[0].hasOwnProperty('devtoolsFrontendUrl'));
|
||||
assert.ok(!response[0].hasOwnProperty('webSocketDebuggerUrl'));
|
||||
}
|
||||
|
||||
function assertScopeValues({ result }, expected) {
|
||||
const unmatched = new Set(Object.keys(expected));
|
||||
for (const actual of result) {
|
||||
@ -290,7 +284,6 @@ async function runTest() {
|
||||
await child.httpGet(null, '/json/badpath').catch(checkBadPath);
|
||||
|
||||
const session = await child.connectInspectorSession();
|
||||
assertNoUrlsWhileConnected(await child.httpGet(null, '/json/list'));
|
||||
await testBreakpointOnStart(session);
|
||||
await testBreakpoint(session);
|
||||
await testI18NCharacters(session);
|
||||
|
Loading…
x
Reference in New Issue
Block a user