Add support for scanning evaluation context from smart cells (#1041)

* Add support for scanning evaluation context from smart cells

* Rename prev_locator to base_locator to clarify the meaning

* Adjustments

* Add data tests

* Adjustments

* Test smart cell intearaction with RuntimeServer

* Serialize binding scanning and leave sending to the user

* Monitor the scanning process
This commit is contained in:
Jonatan Kłosko 2022-03-05 13:19:42 +01:00 committed by GitHub
parent fe53c6a852
commit 6b78258713
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
18 changed files with 648 additions and 141 deletions

View file

@ -208,6 +208,10 @@ defprotocol Livebook.Runtime do
} }
} }
@type smart_cell_ref :: String.t()
@type smart_cell_attrs :: map()
@doc """ @doc """
Connects the caller to the given runtime. Connects the caller to the given runtime.
@ -245,7 +249,7 @@ defprotocol Livebook.Runtime do
be evaluated as well as the evaluation reference to store the be evaluated as well as the evaluation reference to store the
resulting context under. resulting context under.
Additionally, `prev_locator` points to a previous evaluation to be Additionally, `base_locator` points to a previous evaluation to be
used as the starting point of this evaluation. If not applicable, used as the starting point of this evaluation. If not applicable,
the previous evaluation reference may be specified as `nil`. the previous evaluation reference may be specified as `nil`.
@ -284,7 +288,7 @@ defprotocol Livebook.Runtime do
the value of `__DIR__` the value of `__DIR__`
""" """
@spec evaluate_code(t(), String.t(), locator(), locator(), keyword()) :: :ok @spec evaluate_code(t(), String.t(), locator(), locator(), keyword()) :: :ok
def evaluate_code(runtime, code, locator, prev_locator, opts \\ []) def evaluate_code(runtime, code, locator, base_locator, opts \\ [])
@doc """ @doc """
Disposes of an evaluation identified by the given locator. Disposes of an evaluation identified by the given locator.
@ -314,11 +318,11 @@ defprotocol Livebook.Runtime do
* `{:runtime_intellisense_response, ref, request, response}`. * `{:runtime_intellisense_response, ref, request, response}`.
The given `locator` idenfities an evaluation that may be used The given `base_locator` idenfities an evaluation that may be
as the context when resolving the request (if relevant). used as the context when resolving the request (if relevant).
""" """
@spec handle_intellisense(t(), pid(), reference(), intellisense_request(), locator()) :: :ok @spec handle_intellisense(t(), pid(), reference(), intellisense_request(), locator()) :: :ok
def handle_intellisense(runtime, send_to, ref, request, locator) def handle_intellisense(runtime, send_to, ref, request, base_locator)
@doc """ @doc """
Synchronously starts a runtime of the same type with the same Synchronously starts a runtime of the same type with the same
@ -352,6 +356,12 @@ defprotocol Livebook.Runtime do
version of the generated source code. The given `ref` is used to version of the generated source code. The given `ref` is used to
identify the cell. identify the cell.
The cell may depend on evaluation context to provide a better user
experience, for instance it may suggest relevant variable names.
Similarly to `evaluate_code/5`, `base_locator` must be specified
pointing to the evaluation to use as the context. When the locator
changes, it can be updated with `set_smart_cell_base_locator/3`.
Once the cell starts, the runtime sends the following message Once the cell starts, the runtime sends the following message
* `{:runtime_smart_cell_started, ref, %{js_view: js_view(), source: String.t()}}` * `{:runtime_smart_cell_started, ref, %{js_view: js_view(), source: String.t()}}`
@ -368,12 +378,20 @@ defprotocol Livebook.Runtime do
state later. Note that for persistence they get serialized and state later. Note that for persistence they get serialized and
deserialized as JSON. deserialized as JSON.
""" """
@spec start_smart_cell(t(), String.t(), String.t(), term()) :: :ok @spec start_smart_cell(t(), String.t(), smart_cell_ref(), smart_cell_attrs(), locator()) :: :ok
def start_smart_cell(runtime, kind, ref, attrs) def start_smart_cell(runtime, kind, ref, attrs, base_locator)
@doc """
Updates the locator used by a smart cell as its context.
See `start_smart_cell/5` for more details.
"""
@spec set_smart_cell_base_locator(t(), smart_cell_ref(), locator()) :: :ok
def set_smart_cell_base_locator(runtime, ref, base_locator)
@doc """ @doc """
Stops smart cell identified by the given reference. Stops smart cell identified by the given reference.
""" """
@spec stop_smart_cell(t(), String.t()) :: :ok @spec stop_smart_cell(t(), smart_cell_ref()) :: :ok
def stop_smart_cell(runtime, ref) def stop_smart_cell(runtime, ref)
end end

View file

@ -50,8 +50,8 @@ defimpl Livebook.Runtime, for: Livebook.Runtime.Attached do
ErlDist.RuntimeServer.stop(runtime.server_pid) ErlDist.RuntimeServer.stop(runtime.server_pid)
end end
def evaluate_code(runtime, code, locator, prev_locator, opts \\ []) do def evaluate_code(runtime, code, locator, base_locator, opts \\ []) do
ErlDist.RuntimeServer.evaluate_code(runtime.server_pid, code, locator, prev_locator, opts) ErlDist.RuntimeServer.evaluate_code(runtime.server_pid, code, locator, base_locator, opts)
end end
def forget_evaluation(runtime, locator) do def forget_evaluation(runtime, locator) do
@ -62,8 +62,14 @@ defimpl Livebook.Runtime, for: Livebook.Runtime.Attached do
ErlDist.RuntimeServer.drop_container(runtime.server_pid, container_ref) ErlDist.RuntimeServer.drop_container(runtime.server_pid, container_ref)
end end
def handle_intellisense(runtime, send_to, ref, request, locator) do def handle_intellisense(runtime, send_to, ref, request, base_locator) do
ErlDist.RuntimeServer.handle_intellisense(runtime.server_pid, send_to, ref, request, locator) ErlDist.RuntimeServer.handle_intellisense(
runtime.server_pid,
send_to,
ref,
request,
base_locator
)
end end
def duplicate(runtime) do def duplicate(runtime) do
@ -79,8 +85,12 @@ defimpl Livebook.Runtime, for: Livebook.Runtime.Attached do
ErlDist.RuntimeServer.read_file(runtime.server_pid, path) ErlDist.RuntimeServer.read_file(runtime.server_pid, path)
end end
def start_smart_cell(runtime, kind, ref, attrs) do def start_smart_cell(runtime, kind, ref, attrs, base_locator) do
ErlDist.RuntimeServer.start_smart_cell(runtime.server_pid, kind, ref, attrs) ErlDist.RuntimeServer.start_smart_cell(runtime.server_pid, kind, ref, attrs, base_locator)
end
def set_smart_cell_base_locator(runtime, ref, base_locator) do
ErlDist.RuntimeServer.set_smart_cell_base_locator(runtime.server_pid, ref, base_locator)
end end
def stop_smart_cell(runtime, ref) do def stop_smart_cell(runtime, ref) do

View file

@ -78,8 +78,8 @@ defimpl Livebook.Runtime, for: Livebook.Runtime.ElixirStandalone do
ErlDist.RuntimeServer.stop(runtime.server_pid) ErlDist.RuntimeServer.stop(runtime.server_pid)
end end
def evaluate_code(runtime, code, locator, prev_locator, opts \\ []) do def evaluate_code(runtime, code, locator, base_locator, opts \\ []) do
ErlDist.RuntimeServer.evaluate_code(runtime.server_pid, code, locator, prev_locator, opts) ErlDist.RuntimeServer.evaluate_code(runtime.server_pid, code, locator, base_locator, opts)
end end
def forget_evaluation(runtime, locator) do def forget_evaluation(runtime, locator) do
@ -90,8 +90,14 @@ defimpl Livebook.Runtime, for: Livebook.Runtime.ElixirStandalone do
ErlDist.RuntimeServer.drop_container(runtime.server_pid, container_ref) ErlDist.RuntimeServer.drop_container(runtime.server_pid, container_ref)
end end
def handle_intellisense(runtime, send_to, ref, request, locator) do def handle_intellisense(runtime, send_to, ref, request, base_locator) do
ErlDist.RuntimeServer.handle_intellisense(runtime.server_pid, send_to, ref, request, locator) ErlDist.RuntimeServer.handle_intellisense(
runtime.server_pid,
send_to,
ref,
request,
base_locator
)
end end
def duplicate(_runtime) do def duplicate(_runtime) do
@ -104,8 +110,12 @@ defimpl Livebook.Runtime, for: Livebook.Runtime.ElixirStandalone do
ErlDist.RuntimeServer.read_file(runtime.server_pid, path) ErlDist.RuntimeServer.read_file(runtime.server_pid, path)
end end
def start_smart_cell(runtime, kind, ref, attrs) do def start_smart_cell(runtime, kind, ref, attrs, base_locator) do
ErlDist.RuntimeServer.start_smart_cell(runtime.server_pid, kind, ref, attrs) ErlDist.RuntimeServer.start_smart_cell(runtime.server_pid, kind, ref, attrs, base_locator)
end
def set_smart_cell_base_locator(runtime, ref, base_locator) do
ErlDist.RuntimeServer.set_smart_cell_base_locator(runtime.server_pid, ref, base_locator)
end end
def stop_smart_cell(runtime, ref) do def stop_smart_cell(runtime, ref) do

View file

@ -51,8 +51,8 @@ defimpl Livebook.Runtime, for: Livebook.Runtime.Embedded do
ErlDist.RuntimeServer.stop(runtime.server_pid) ErlDist.RuntimeServer.stop(runtime.server_pid)
end end
def evaluate_code(runtime, code, locator, prev_locator, opts \\ []) do def evaluate_code(runtime, code, locator, base_locator, opts \\ []) do
ErlDist.RuntimeServer.evaluate_code(runtime.server_pid, code, locator, prev_locator, opts) ErlDist.RuntimeServer.evaluate_code(runtime.server_pid, code, locator, base_locator, opts)
end end
def forget_evaluation(runtime, locator) do def forget_evaluation(runtime, locator) do
@ -63,8 +63,14 @@ defimpl Livebook.Runtime, for: Livebook.Runtime.Embedded do
ErlDist.RuntimeServer.drop_container(runtime.server_pid, container_ref) ErlDist.RuntimeServer.drop_container(runtime.server_pid, container_ref)
end end
def handle_intellisense(runtime, send_to, ref, request, locator) do def handle_intellisense(runtime, send_to, ref, request, base_locator) do
ErlDist.RuntimeServer.handle_intellisense(runtime.server_pid, send_to, ref, request, locator) ErlDist.RuntimeServer.handle_intellisense(
runtime.server_pid,
send_to,
ref,
request,
base_locator
)
end end
def duplicate(_runtime) do def duplicate(_runtime) do
@ -77,8 +83,12 @@ defimpl Livebook.Runtime, for: Livebook.Runtime.Embedded do
ErlDist.RuntimeServer.read_file(runtime.server_pid, path) ErlDist.RuntimeServer.read_file(runtime.server_pid, path)
end end
def start_smart_cell(runtime, kind, ref, attrs) do def start_smart_cell(runtime, kind, ref, attrs, base_locator) do
ErlDist.RuntimeServer.start_smart_cell(runtime.server_pid, kind, ref, attrs) ErlDist.RuntimeServer.start_smart_cell(runtime.server_pid, kind, ref, attrs, base_locator)
end
def set_smart_cell_base_locator(runtime, ref, base_locator) do
ErlDist.RuntimeServer.set_smart_cell_base_locator(runtime.server_pid, ref, base_locator)
end end
def stop_smart_cell(runtime, ref) do def stop_smart_cell(runtime, ref) do

View file

@ -71,9 +71,9 @@ defmodule Livebook.Runtime.ErlDist.NodeManager do
@doc """ @doc """
Starts a new `Livebook.Runtime.ErlDist.RuntimeServer` for evaluation. Starts a new `Livebook.Runtime.ErlDist.RuntimeServer` for evaluation.
""" """
@spec start_runtime_server(node() | pid()) :: pid() @spec start_runtime_server(node() | pid(), keyword()) :: pid()
def start_runtime_server(node_or_pid) do def start_runtime_server(node_or_pid, opts \\ []) do
GenServer.call(server(node_or_pid), :start_runtime_server) GenServer.call(server(node_or_pid), {:start_runtime_server, opts})
end end
defp server(pid) when is_pid(pid), do: pid defp server(pid) when is_pid(pid), do: pid
@ -154,9 +154,9 @@ defmodule Livebook.Runtime.ErlDist.NodeManager do
def handle_info(_message, state), do: {:noreply, state} def handle_info(_message, state), do: {:noreply, state}
@impl true @impl true
def handle_call(:start_runtime_server, _from, state) do def handle_call({:start_runtime_server, opts}, _from, state) do
{:ok, server_pid} = {:ok, server_pid} =
DynamicSupervisor.start_child(state.server_supevisor, ErlDist.RuntimeServer) DynamicSupervisor.start_child(state.server_supevisor, {ErlDist.RuntimeServer, opts})
Process.monitor(server_pid) Process.monitor(server_pid)
state = update_in(state.runtime_servers, &[server_pid | &1]) state = update_in(state.runtime_servers, &[server_pid | &1])

View file

@ -15,6 +15,8 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServer do
use GenServer, restart: :temporary use GenServer, restart: :temporary
require Logger
alias Livebook.Runtime.Evaluator alias Livebook.Runtime.Evaluator
alias Livebook.Runtime alias Livebook.Runtime
alias Livebook.Runtime.ErlDist alias Livebook.Runtime.ErlDist
@ -27,6 +29,12 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServer do
Note: make sure to call `attach` within #{@await_owner_timeout}ms Note: make sure to call `attach` within #{@await_owner_timeout}ms
or the runtime server assumes it's not needed and terminates. or the runtime server assumes it's not needed and terminates.
## Options
* `:smart_cell_definitions_module` - the module to read smart
cell definitions from, it needs to export a `definitions/0`
function. Defaults to `Kino.SmartCell`
""" """
def start_link(opts \\ []) do def start_link(opts \\ []) do
GenServer.start_link(__MODULE__, opts) GenServer.start_link(__MODULE__, opts)
@ -60,8 +68,8 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServer do
See `Livebook.Runtime.Evaluator` for more details. See `Livebook.Runtime.Evaluator` for more details.
""" """
@spec evaluate_code(pid(), String.t(), Runtime.locator(), Runtime.locator(), keyword()) :: :ok @spec evaluate_code(pid(), String.t(), Runtime.locator(), Runtime.locator(), keyword()) :: :ok
def evaluate_code(pid, code, locator, prev_locator, opts \\ []) do def evaluate_code(pid, code, locator, base_locator, opts \\ []) do
GenServer.cast(pid, {:evaluate_code, code, locator, prev_locator, opts}) GenServer.cast(pid, {:evaluate_code, code, locator, base_locator, opts})
end end
@doc """ @doc """
@ -100,8 +108,8 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServer do
Runtime.intellisense_request(), Runtime.intellisense_request(),
Runtime.locator() Runtime.locator()
) :: :ok ) :: :ok
def handle_intellisense(pid, send_to, ref, request, locator) do def handle_intellisense(pid, send_to, ref, request, base_locator) do
GenServer.cast(pid, {:handle_intellisense, send_to, ref, request, locator}) GenServer.cast(pid, {:handle_intellisense, send_to, ref, request, base_locator})
end end
@doc """ @doc """
@ -126,9 +134,23 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServer do
@doc """ @doc """
Starts a new smart cell. Starts a new smart cell.
""" """
@spec start_smart_cell(pid(), String.t(), String.t(), term()) :: :ok @spec start_smart_cell(
def start_smart_cell(pid, kind, ref, attrs) do pid(),
GenServer.cast(pid, {:start_smart_cell, kind, ref, attrs}) String.t(),
Runtime.smart_cell_ref(),
Runtime.smart_cell_attrs(),
Runtime.locator()
) :: :ok
def start_smart_cell(pid, kind, ref, attrs, base_locator) do
GenServer.cast(pid, {:start_smart_cell, kind, ref, attrs, base_locator})
end
@doc """
Updates the locator with smart cell context.
"""
@spec set_smart_cell_base_locator(pid(), Runtime.smart_cell_ref(), Runtime.locator()) :: :ok
def set_smart_cell_base_locator(pid, ref, base_locator) do
GenServer.cast(pid, {:set_smart_cell_base_locator, ref, base_locator})
end end
@doc """ @doc """
@ -151,7 +173,7 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServer do
end end
@impl true @impl true
def init(_opts) do def init(opts) do
Process.send_after(self(), :check_owner, @await_owner_timeout) Process.send_after(self(), :check_owner, @await_owner_timeout)
schedule_memory_usage_report() schedule_memory_usage_report()
@ -171,6 +193,8 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServer do
smart_cell_gl: nil, smart_cell_gl: nil,
smart_cells: %{}, smart_cells: %{},
smart_cell_definitions: [], smart_cell_definitions: [],
smart_cell_definitions_module:
Keyword.get(opts, :smart_cell_definitions_module, Kino.SmartCell),
memory_timer_ref: nil memory_timer_ref: nil
}} }}
end end
@ -190,24 +214,18 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServer do
{:stop, :shutdown, state} {:stop, :shutdown, state}
end end
def handle_info({:DOWN, _, :process, pid, reason}, state) do def handle_info({:DOWN, _, :process, _, _} = message, state) do
state.evaluators {:noreply,
|> Enum.find(fn {_container_ref, evaluator} -> state
evaluator.pid == pid |> handle_down_evaluator(message)
end) |> handle_down_scan_binding(message)}
|> case do
{container_ref, _} ->
message = Exception.format_exit(reason)
send(state.owner, {:runtime_container_down, container_ref, message})
{:noreply, %{state | evaluators: Map.delete(state.evaluators, container_ref)}}
nil ->
{:noreply, state}
end
end end
def handle_info({:evaluation_finished, _ref}, state) do def handle_info({:evaluation_finished, pid, evaluation_ref}, state) do
{:noreply, report_smart_cell_definitions(state)} {:noreply,
state
|> report_smart_cell_definitions()
|> scan_binding_after_evaluation(pid, evaluation_ref)}
end end
def handle_info(:memory_usage, state) do def handle_info(:memory_usage, state) do
@ -216,8 +234,37 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServer do
{:noreply, state} {:noreply, state}
end end
def handle_info({:scan_binding_ack, ref}, state) do
{:noreply, finish_scan_binding(ref, state)}
end
def handle_info(_message, state), do: {:noreply, state} def handle_info(_message, state), do: {:noreply, state}
defp handle_down_evaluator(state, {:DOWN, _, :process, pid, reason}) do
state.evaluators
|> Enum.find(fn {_container_ref, evaluator} -> evaluator.pid == pid end)
|> case do
{container_ref, _} ->
message = Exception.format_exit(reason)
send(state.owner, {:runtime_container_down, container_ref, message})
%{state | evaluators: Map.delete(state.evaluators, container_ref)}
nil ->
state
end
end
defp handle_down_scan_binding(state, {:DOWN, monitor_ref, :process, _, _}) do
Enum.find_value(state.smart_cells, fn
{ref, %{scan_binding_monitor_ref: ^monitor_ref}} -> ref
_ -> nil
end)
|> case do
nil -> state
ref -> finish_scan_binding(ref, state)
end
end
@impl true @impl true
def handle_cast({:attach, owner, opts}, state) do def handle_cast({:attach, owner, opts}, state) do
if state.owner do if state.owner do
@ -239,13 +286,13 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServer do
end end
def handle_cast( def handle_cast(
{:evaluate_code, code, {container_ref, evaluation_ref}, prev_locator, opts}, {:evaluate_code, code, {container_ref, evaluation_ref}, base_locator, opts},
state state
) do ) do
state = ensure_evaluator(state, container_ref) state = ensure_evaluator(state, container_ref)
prev_evaluation_ref = base_evaluation_ref =
case prev_locator do case base_locator do
{^container_ref, evaluation_ref} -> {^container_ref, evaluation_ref} ->
evaluation_ref evaluation_ref
@ -265,7 +312,7 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServer do
state.evaluators[container_ref], state.evaluators[container_ref],
code, code,
evaluation_ref, evaluation_ref,
prev_evaluation_ref, base_evaluation_ref,
opts opts
) )
@ -285,8 +332,8 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServer do
{:noreply, state} {:noreply, state}
end end
def handle_cast({:handle_intellisense, send_to, ref, request, locator}, state) do def handle_cast({:handle_intellisense, send_to, ref, request, base_locator}, state) do
{container_ref, evaluation_ref} = locator {container_ref, evaluation_ref} = base_locator
evaluator = state.evaluators[container_ref] evaluator = state.evaluators[container_ref]
intellisense_context = intellisense_context =
@ -304,7 +351,7 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServer do
{:noreply, state} {:noreply, state}
end end
def handle_cast({:start_smart_cell, kind, ref, attrs}, state) do def handle_cast({:start_smart_cell, kind, ref, attrs, base_locator}, state) do
definition = Enum.find(state.smart_cell_definitions, &(&1.kind == kind)) definition = Enum.find(state.smart_cell_definitions, &(&1.kind == kind))
state = state =
@ -313,8 +360,23 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServer do
{definition.module, %{ref: ref, attrs: attrs, target_pid: state.owner}} {definition.module, %{ref: ref, attrs: attrs, target_pid: state.owner}}
) do ) do
{:ok, pid, info} -> {:ok, pid, info} ->
send(state.owner, {:runtime_smart_cell_started, ref, info}) %{js_view: js_view, source: source, scan_binding: scan_binding} = info
put_in(state.smart_cells[ref], pid)
send(
state.owner,
{:runtime_smart_cell_started, ref, %{js_view: js_view, source: source}}
)
info = %{
pid: pid,
scan_binding: scan_binding,
base_locator: base_locator,
scan_binding_pending: false,
scan_binding_monitor_ref: nil
}
info = scan_binding_async(ref, info, state)
put_in(state.smart_cells[ref], info)
_ -> _ ->
state state
@ -323,8 +385,18 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServer do
{:noreply, state} {:noreply, state}
end end
def handle_cast({:set_smart_cell_base_locator, ref, base_locator}, state) do
state =
update_in(state.smart_cells[ref], fn
%{base_locator: ^base_locator} = info -> info
info -> scan_binding_async(ref, %{info | base_locator: base_locator}, state)
end)
{:noreply, state}
end
def handle_cast({:stop_smart_cell, ref}, state) do def handle_cast({:stop_smart_cell, ref}, state) do
{pid, state} = pop_in(state.smart_cells[ref]) {%{pid: pid}, state} = pop_in(state.smart_cells[ref])
if pid do if pid do
DynamicSupervisor.terminate_child(state.smart_cell_supervisor, pid) DynamicSupervisor.terminate_child(state.smart_cell_supervisor, pid)
@ -393,7 +465,7 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServer do
end end
defp report_smart_cell_definitions(state) do defp report_smart_cell_definitions(state) do
smart_cell_definitions = get_smart_cell_definitions() smart_cell_definitions = get_smart_cell_definitions(state.smart_cell_definitions_module)
if smart_cell_definitions == state.smart_cell_definitions do if smart_cell_definitions == state.smart_cell_definitions do
state state
@ -404,13 +476,84 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServer do
end end
end end
@compile {:no_warn_undefined, {Kino.SmartCell, :definitions, 0}} defp get_smart_cell_definitions(module) do
if Code.ensure_loaded?(module) and function_exported?(module, :definitions, 0) do
defp get_smart_cell_definitions() do module.definitions()
if Code.ensure_loaded?(Kino.SmartCell) and function_exported?(Kino.SmartCell, :definitions, 0) do
Kino.SmartCell.definitions()
else else
[] []
end end
end end
defp scan_binding_async(_ref, %{scan_binding: nil} = info, _state), do: info
# We wait for the current scanning to finish, this way we avoid
# race conditions and don't unnecessarily spam evaluators
defp scan_binding_async(_ref, %{scan_binding_monitor_ref: ref} = info, _state) when ref != nil,
do: %{info | scan_binding_pending: true}
defp scan_binding_async(ref, info, state) do
%{pid: pid, scan_binding: scan_binding} = info
myself = self()
scan_and_ack = fn binding, env ->
try do
scan_binding.(pid, binding, env)
rescue
error -> Logger.error("scanning binding raised an error: #{inspect(error)}")
end
send(myself, {:scan_binding_ack, ref})
end
{container_ref, evaluation_ref} = info.base_locator
evaluator = state.evaluators[container_ref]
worker_pid =
if evaluator do
Evaluator.peek_context(evaluator, evaluation_ref, &scan_and_ack.(&1.binding, &1.env))
evaluator.pid
else
{:ok, pid} =
Task.Supervisor.start_child(state.task_supervisor, fn ->
binding = []
# TODO: Use Code.env_for_eval and eval_quoted_with_env on Elixir v1.14+
env = :elixir.env_for_eval([])
scan_and_ack.(binding, env)
end)
pid
end
monitor_ref = Process.monitor(worker_pid)
%{info | scan_binding_pending: false, scan_binding_monitor_ref: monitor_ref}
end
defp finish_scan_binding(ref, state) do
update_in(state.smart_cells[ref], fn info ->
Process.demonitor(info.scan_binding_monitor_ref, [:flush])
info = %{info | scan_binding_monitor_ref: nil}
if info.scan_binding_pending do
scan_binding_async(ref, info, state)
else
info
end
end)
end
defp scan_binding_after_evaluation(state, pid, evaluation_ref) do
{container_ref, _} =
Enum.find(state.evaluators, fn {_container_ref, evaluator} -> evaluator.pid == pid end)
locator = {container_ref, evaluation_ref}
update_in(state.smart_cells, fn smart_cells ->
Map.map(smart_cells, fn
{ref, %{base_locator: ^locator} = info} -> scan_binding_async(ref, info, state)
{_, info} -> info
end)
end)
end
end end

View file

@ -118,7 +118,7 @@ defmodule Livebook.Runtime.Evaluator do
response. response.
The resulting contxt (binding and env) is stored under `ref`. The resulting contxt (binding and env) is stored under `ref`.
Any subsequent calls may specify `prev_ref` pointing to a Any subsequent calls may specify `base_ref` pointing to a
previous evaluation, in which case the corresponding context previous evaluation, in which case the corresponding context
is used as the entry point for evaluation. is used as the entry point for evaluation.
@ -132,11 +132,11 @@ defmodule Livebook.Runtime.Evaluator do
* `:notify_to` - a process to be notified about finished * `:notify_to` - a process to be notified about finished
evaluation. The notification is sent as a message of the evaluation. The notification is sent as a message of the
form `{:evaluation_finished, ref}` form `{:evaluation_finished, pid, ref}`
""" """
@spec evaluate_code(t(), String.t(), ref(), ref() | nil, keyword()) :: :ok @spec evaluate_code(t(), String.t(), ref(), ref() | nil, keyword()) :: :ok
def evaluate_code(evaluator, code, ref, prev_ref \\ nil, opts \\ []) when ref != nil do def evaluate_code(evaluator, code, ref, base_ref \\ nil, opts \\ []) when ref != nil do
cast(evaluator, {:evaluate_code, code, ref, prev_ref, opts}) cast(evaluator, {:evaluate_code, code, ref, base_ref, opts})
end end
@doc """ @doc """
@ -216,6 +216,17 @@ defmodule Livebook.Runtime.Evaluator do
call(evaluator, {:map_binding, ref, fun}) call(evaluator, {:map_binding, ref, fun})
end end
@doc """
Runs the given function with binding and env of the given evaluation.
Ths function runs within the evaluator process, so that no data
is copied between processes, unless explicitly sent.
"""
@spec peek_context(t(), ref(), (context() -> any())) :: :ok
def peek_context(evaluator, ref, fun) do
cast(evaluator, {:peek_context, ref, fun})
end
defp cast(evaluator, message) do defp cast(evaluator, message) do
send(evaluator.pid, {:cast, evaluator.ref, message}) send(evaluator.pid, {:cast, evaluator.ref, message})
:ok :ok
@ -298,12 +309,12 @@ defmodule Livebook.Runtime.Evaluator do
%{binding: [], env: env, id: random_id()} %{binding: [], env: env, id: random_id()}
end end
defp handle_cast({:evaluate_code, code, ref, prev_ref, opts}, state) do defp handle_cast({:evaluate_code, code, ref, base_ref, opts}, state) do
Evaluator.IOProxy.configure(state.io_proxy, ref) Evaluator.IOProxy.configure(state.io_proxy, ref)
Evaluator.ObjectTracker.remove_reference(state.object_tracker, {self(), ref}) Evaluator.ObjectTracker.remove_reference(state.object_tracker, {self(), ref})
context = get_context(state, prev_ref) context = get_context(state, base_ref)
file = Keyword.get(opts, :file, "nofile") file = Keyword.get(opts, :file, "nofile")
context = put_in(context.env.file, file) context = put_in(context.env.file, file)
start_time = System.monotonic_time() start_time = System.monotonic_time()
@ -338,7 +349,7 @@ defmodule Livebook.Runtime.Evaluator do
send(state.send_to, {:runtime_evaluation_response, ref, output, metadata}) send(state.send_to, {:runtime_evaluation_response, ref, output, metadata})
if notify_to = opts[:notify_to] do if notify_to = opts[:notify_to] do
send(notify_to, {:evaluation_finished, ref}) send(notify_to, {:evaluation_finished, self(), ref})
end end
:erlang.garbage_collect(self()) :erlang.garbage_collect(self())
@ -353,6 +364,12 @@ defmodule Livebook.Runtime.Evaluator do
{:noreply, state} {:noreply, state}
end end
defp handle_cast({:peek_context, ref, fun}, state) do
context = get_context(state, ref)
fun.(context)
{:noreply, state}
end
defp handle_call({:fetch_evaluation_context, ref, cached_id}, _from, state) do defp handle_call({:fetch_evaluation_context, ref, cached_id}, _from, state) do
context = get_context(state, ref) context = get_context(state, ref)

View file

@ -145,8 +145,8 @@ defimpl Livebook.Runtime, for: Livebook.Runtime.MixStandalone do
ErlDist.RuntimeServer.stop(runtime.server_pid) ErlDist.RuntimeServer.stop(runtime.server_pid)
end end
def evaluate_code(runtime, code, locator, prev_locator, opts \\ []) do def evaluate_code(runtime, code, locator, base_locator, opts \\ []) do
ErlDist.RuntimeServer.evaluate_code(runtime.server_pid, code, locator, prev_locator, opts) ErlDist.RuntimeServer.evaluate_code(runtime.server_pid, code, locator, base_locator, opts)
end end
def forget_evaluation(runtime, locator) do def forget_evaluation(runtime, locator) do
@ -157,8 +157,14 @@ defimpl Livebook.Runtime, for: Livebook.Runtime.MixStandalone do
ErlDist.RuntimeServer.drop_container(runtime.server_pid, container_ref) ErlDist.RuntimeServer.drop_container(runtime.server_pid, container_ref)
end end
def handle_intellisense(runtime, send_to, ref, request, locator) do def handle_intellisense(runtime, send_to, ref, request, base_locator) do
ErlDist.RuntimeServer.handle_intellisense(runtime.server_pid, send_to, ref, request, locator) ErlDist.RuntimeServer.handle_intellisense(
runtime.server_pid,
send_to,
ref,
request,
base_locator
)
end end
def duplicate(runtime) do def duplicate(runtime) do
@ -171,8 +177,12 @@ defimpl Livebook.Runtime, for: Livebook.Runtime.MixStandalone do
ErlDist.RuntimeServer.read_file(runtime.server_pid, path) ErlDist.RuntimeServer.read_file(runtime.server_pid, path)
end end
def start_smart_cell(runtime, kind, ref, attrs) do def start_smart_cell(runtime, kind, ref, attrs, base_locator) do
ErlDist.RuntimeServer.start_smart_cell(runtime.server_pid, kind, ref, attrs) ErlDist.RuntimeServer.start_smart_cell(runtime.server_pid, kind, ref, attrs, base_locator)
end
def set_smart_cell_base_locator(runtime, ref, base_locator) do
ErlDist.RuntimeServer.set_smart_cell_base_locator(runtime.server_pid, ref, base_locator)
end end
def stop_smart_cell(runtime, ref) do def stop_smart_cell(runtime, ref) do

View file

@ -56,6 +56,7 @@ defmodule Livebook.Session do
alias Livebook.Notebook.{Cell, Section} alias Livebook.Notebook.{Cell, Section}
@timeout :infinity @timeout :infinity
@main_container_ref :main_flow
@type t :: %__MODULE__{ @type t :: %__MODULE__{
id: id(), id: id(),
@ -879,7 +880,7 @@ defmodule Livebook.Session do
operation = operation =
case container_ref do case container_ref do
:main_flow -> {:reflect_main_evaluation_failure, self()} @main_container_ref -> {:reflect_main_evaluation_failure, self()}
section_id -> {:reflect_evaluation_failure, self(), section_id} section_id -> {:reflect_evaluation_failure, self(), section_id}
end end
@ -1211,8 +1212,8 @@ defmodule Livebook.Session do
opts = [file: file] opts = [file: file]
locator = {container_ref_for_section(section), cell.id} locator = {container_ref_for_section(section), cell.id}
prev_locator = find_prev_locator(state.data.notebook, cell, section) base_locator = find_base_locator(state.data, cell, section)
Runtime.evaluate_code(state.data.runtime, cell.source, locator, prev_locator, opts) Runtime.evaluate_code(state.data.runtime, cell.source, locator, base_locator, opts)
evaluation_digest = :erlang.md5(cell.source) evaluation_digest = :erlang.md5(cell.source)
handle_operation(state, {:evaluation_started, self(), cell.id, evaluation_digest}) handle_operation(state, {:evaluation_started, self(), cell.id, evaluation_digest})
@ -1234,9 +1235,27 @@ defmodule Livebook.Session do
state state
end end
defp handle_action(state, {:start_smart_cell, cell}) do defp handle_action(state, {:start_smart_cell, cell, section}) do
if state.data.runtime do if state.data.runtime do
Runtime.start_smart_cell(state.data.runtime, cell.kind, cell.id, cell.attrs) base_locator = find_base_locator(state.data, cell, section, existing: true)
Runtime.start_smart_cell(state.data.runtime, cell.kind, cell.id, cell.attrs, base_locator)
end
state
end
defp handle_action(state, {:set_smart_cell_base, cell, section, parent}) do
if state.data.runtime do
base_locator =
case parent do
nil ->
{container_ref_for_section(section), nil}
{parent_cell, parent_section} ->
{container_ref_for_section(parent_section), parent_cell.id}
end
Runtime.set_smart_cell_base_locator(state.data.runtime, cell.id, base_locator)
end end
state state
@ -1461,20 +1480,36 @@ defmodule Livebook.Session do
end end
@doc """ @doc """
Determines locator of the evaluation that the given Finds evaluation locator that the given cell depends on.
cell depends on.
By default looks up the direct evaluation parent.
## Options
* `:existing` - considers only cells that have been evaluated
as evaluation parents. Defaults to `false`
""" """
@spec find_prev_locator(Notebook.t(), Cell.t(), Section.t()) :: Runtime.locator() @spec find_base_locator(Data.t(), Cell.t(), Section.t(), keyword()) :: Runtime.locator()
def find_prev_locator(notebook, cell, section) do def find_base_locator(data, cell, section, opts \\ []) do
parent_filter =
if opts[:existing] do
fn cell ->
info = data.cell_infos[cell.id]
Cell.evaluable?(cell) and info.eval.validity in [:evaluated, :stale]
end
else
&Cell.evaluable?/1
end
default = {container_ref_for_section(section), nil} default = {container_ref_for_section(section), nil}
notebook data.notebook
|> Notebook.parent_cells_with_section(cell.id) |> Notebook.parent_cells_with_section(cell.id)
|> Enum.find_value(default, fn {cell, section} -> |> Enum.find_value(default, fn {cell, section} ->
Cell.evaluable?(cell) && {container_ref_for_section(section), cell.id} parent_filter.(cell) && {container_ref_for_section(section), cell.id}
end) end)
end end
defp container_ref_for_section(%{parent_id: nil}), do: :main_flow defp container_ref_for_section(%{parent_id: nil}), do: @main_container_ref
defp container_ref_for_section(section), do: section.id defp container_ref_for_section(section), do: section.id
end end

View file

@ -108,7 +108,7 @@ defmodule Livebook.Session.Data do
@type cell_evaluation_validity :: :fresh | :evaluated | :stale | :aborted @type cell_evaluation_validity :: :fresh | :evaluated | :stale | :aborted
@type cell_evaluation_status :: :ready | :queued | :evaluating @type cell_evaluation_status :: :ready | :queued | :evaluating
@type smart_cell_status :: :dead | :starting | :alive @type smart_cell_status :: :dead | :starting | :started
@type input_id :: String.t() @type input_id :: String.t()
@ -186,7 +186,8 @@ defmodule Livebook.Session.Data do
| {:start_evaluation, Cell.t(), Section.t()} | {:start_evaluation, Cell.t(), Section.t()}
| {:stop_evaluation, Section.t()} | {:stop_evaluation, Section.t()}
| {:forget_evaluation, Cell.t(), Section.t()} | {:forget_evaluation, Cell.t(), Section.t()}
| {:start_smart_cell, Cell.t()} | {:start_smart_cell, Cell.t(), Section.t()}
| {:set_smart_cell_base, Cell.t(), Section.t(), parent :: {Cell.t(), Section.t()} | nil}
| {:broadcast_delta, pid(), Cell.t(), Delta.t()} | {:broadcast_delta, pid(), Cell.t(), Delta.t()}
@doc """ @doc """
@ -358,6 +359,7 @@ defmodule Livebook.Session.Data do
|> with_actions() |> with_actions()
|> delete_section(section, delete_cells) |> delete_section(section, delete_cells)
|> compute_snapshots_and_validity() |> compute_snapshots_and_validity()
|> update_smart_cell_bases(data)
|> set_dirty() |> set_dirty()
|> wrap_ok() |> wrap_ok()
else else
@ -371,6 +373,7 @@ defmodule Livebook.Session.Data do
|> with_actions() |> with_actions()
|> delete_cell(cell, section) |> delete_cell(cell, section)
|> compute_snapshots_and_validity() |> compute_snapshots_and_validity()
|> update_smart_cell_bases(data)
|> set_dirty() |> set_dirty()
|> wrap_ok() |> wrap_ok()
end end
@ -398,6 +401,7 @@ defmodule Livebook.Session.Data do
|> with_actions() |> with_actions()
|> move_cell(cell, offset) |> move_cell(cell, offset)
|> compute_snapshots_and_validity() |> compute_snapshots_and_validity()
|> update_smart_cell_bases(data)
|> set_dirty() |> set_dirty()
|> wrap_ok() |> wrap_ok()
else else
@ -413,6 +417,7 @@ defmodule Livebook.Session.Data do
|> with_actions() |> with_actions()
|> move_section(section, offset) |> move_section(section, offset)
|> compute_snapshots_and_validity() |> compute_snapshots_and_validity()
|> update_smart_cell_bases(data)
|> set_dirty() |> set_dirty()
|> wrap_ok() |> wrap_ok()
else else
@ -482,6 +487,7 @@ defmodule Livebook.Session.Data do
|> compute_snapshots_and_validity() |> compute_snapshots_and_validity()
|> maybe_evaluate_queued() |> maybe_evaluate_queued()
|> compute_snapshots_and_validity() |> compute_snapshots_and_validity()
|> update_smart_cell_bases(data)
|> mark_dirty_if_persisting_outputs() |> mark_dirty_if_persisting_outputs()
|> wrap_ok() |> wrap_ok()
else else
@ -507,6 +513,7 @@ defmodule Livebook.Session.Data do
data data
|> with_actions() |> with_actions()
|> clear_main_evaluation() |> clear_main_evaluation()
|> update_smart_cell_bases(data)
|> wrap_ok() |> wrap_ok()
end end
@ -515,6 +522,7 @@ defmodule Livebook.Session.Data do
data data
|> with_actions() |> with_actions()
|> clear_section_evaluation(section) |> clear_section_evaluation(section)
|> update_smart_cell_bases(data)
|> wrap_ok() |> wrap_ok()
end end
end end
@ -525,6 +533,7 @@ defmodule Livebook.Session.Data do
data data
|> with_actions() |> with_actions()
|> cancel_cell_evaluation(cell, section) |> cancel_cell_evaluation(cell, section)
|> update_smart_cell_bases(data)
|> wrap_ok() |> wrap_ok()
else else
_ -> :error _ -> :error
@ -563,6 +572,7 @@ defmodule Livebook.Session.Data do
|> with_actions() |> with_actions()
|> erase_outputs() |> erase_outputs()
|> garbage_collect_input_values() |> garbage_collect_input_values()
|> update_smart_cell_bases(data)
|> wrap_ok() |> wrap_ok()
end end
@ -1203,7 +1213,7 @@ defmodule Livebook.Session.Data do
data_actions data_actions
|> set!(notebook: Notebook.update_cell(data.notebook, cell.id, fn _ -> updated_cell end)) |> set!(notebook: Notebook.update_cell(data.notebook, cell.id, fn _ -> updated_cell end))
|> update_cell_info!(cell.id, &%{&1 | status: :alive}) |> update_cell_info!(cell.id, &%{&1 | status: :started})
|> add_action({:broadcast_delta, client_pid, updated_cell, delta}) |> add_action({:broadcast_delta, client_pid, updated_cell, delta})
end end
@ -1367,26 +1377,26 @@ defmodule Livebook.Session.Data do
defp maybe_start_smart_cells({data, _} = data_actions) do defp maybe_start_smart_cells({data, _} = data_actions) do
if data.runtime do if data.runtime do
dead_cells = dead_smart_cells(data) dead_cells = dead_smart_cells_with_section(data)
kinds = Enum.map(data.smart_cell_definitions, & &1.kind) kinds = Enum.map(data.smart_cell_definitions, & &1.kind)
cells_ready_to_start = Enum.filter(dead_cells, &(&1.kind in kinds)) cells_ready_to_start = Enum.filter(dead_cells, fn {cell, _} -> cell.kind in kinds end)
reduce(data_actions, cells_ready_to_start, fn data_actions, cell -> reduce(data_actions, cells_ready_to_start, fn data_actions, {cell, section} ->
data_actions data_actions
|> update_cell_info!(cell.id, &%{&1 | status: :starting}) |> update_cell_info!(cell.id, &%{&1 | status: :starting})
|> add_action({:start_smart_cell, cell}) |> add_action({:start_smart_cell, cell, section})
end) end)
else else
data_actions data_actions
end end
end end
defp dead_smart_cells(data) do defp dead_smart_cells_with_section(data) do
for section <- data.notebook.sections, for section <- data.notebook.sections,
%Cell.Smart{} = cell <- section.cells, %Cell.Smart{} = cell <- section.cells,
info = data.cell_infos[cell.id], info = data.cell_infos[cell.id],
info.status == :dead, info.status == :dead,
do: cell do: {cell, section}
end end
defp clear_smart_cells({data, _} = data_actions) do defp clear_smart_cells({data, _} = data_actions) do
@ -1453,6 +1463,75 @@ defmodule Livebook.Session.Data do
end end
end end
defp update_smart_cell_bases({data, _} = data_actions, prev_data) do
alive_smart_cell_ids =
for {%Cell.Smart{} = cell, _} <- Notebook.cells_with_section(data.notebook),
data.cell_infos[cell.id].status != :dead,
into: MapSet.new(),
do: cell.id
if Enum.empty?(alive_smart_cell_ids) do
data_actions
else
new_eval_graph = cell_evaluation_graph(data)
prev_eval_graph = cell_evaluation_graph(prev_data)
cell_lookup =
data.notebook
|> Notebook.cells_with_section()
|> Map.new(fn {cell, section} -> {cell.id, {cell, section}} end)
for {cell_id, parent_id} <- new_eval_graph,
MapSet.member?(alive_smart_cell_ids, cell_id),
Map.has_key?(prev_eval_graph, cell_id),
prev_eval_graph[cell_id] != parent_id,
reduce: data_actions do
data_actions ->
{cell, section} = cell_lookup[cell_id]
parent = cell_lookup[parent_id]
add_action(data_actions, {:set_smart_cell_base, cell, section, parent})
end
end
end
# Builds a graph with evaluation parents, where each parent has
# aleady been evaluated. All fresh/aborted cells are leaves in
# this graph
defp cell_evaluation_graph(data) do
graph = Notebook.cell_dependency_graph(data.notebook, cell_filter: &Cell.evaluable?/1)
graph
|> Livebook.Utils.Graph.leaves()
|> Enum.reduce(%{}, fn cell_id, eval_graph ->
build_eval_graph(data, graph, cell_id, [], eval_graph)
end)
end
defp build_eval_graph(_data, _graph, nil, orphan_ids, eval_graph) do
put_parent(eval_graph, orphan_ids, nil)
end
defp build_eval_graph(data, graph, cell_id, orphan_ids, eval_graph) do
# We are traversing from every leaf up, so we want to compute
# the common path only once
if eval_parent_id = eval_graph[cell_id] do
put_parent(eval_graph, orphan_ids, eval_parent_id)
else
info = data.cell_infos[cell_id]
if info.eval.validity in [:evaluated, :stale] do
eval_graph = put_parent(eval_graph, orphan_ids, cell_id)
build_eval_graph(data, graph, graph[cell_id], [cell_id], eval_graph)
else
build_eval_graph(data, graph, graph[cell_id], [cell_id | orphan_ids], eval_graph)
end
end
end
defp put_parent(eval_graph, cell_ids, parent_id) do
Enum.reduce(cell_ids, eval_graph, &Map.put(&2, &1, parent_id))
end
defp new_section_info() do defp new_section_info() do
%{ %{
evaluating_cell_id: nil, evaluating_cell_id: nil,
@ -1671,6 +1750,7 @@ defmodule Livebook.Session.Data do
update_cell_eval_info!(data_actions, cell.id, fn eval_info -> update_cell_eval_info!(data_actions, cell.id, fn eval_info ->
validity = validity =
case eval_info do case eval_info do
%{status: :evaluating, validity: validity} -> validity
%{evaluation_snapshot: snapshot, snapshot: snapshot} -> :evaluated %{evaluation_snapshot: snapshot, snapshot: snapshot} -> :evaluated
%{evaluation_snapshot: nil, validity: :aborted} -> :aborted %{evaluation_snapshot: nil, validity: :aborted} -> :aborted
%{evaluation_snapshot: nil} -> :fresh %{evaluation_snapshot: nil} -> :fresh
@ -1688,7 +1768,9 @@ defmodule Livebook.Session.Data do
|> Notebook.evaluable_cells_with_section() |> Notebook.evaluable_cells_with_section()
|> Enum.filter(fn {cell, _section} -> |> Enum.filter(fn {cell, _section} ->
info = data.cell_infos[cell.id] info = data.cell_infos[cell.id]
info.eval.validity == :stale and Map.get(cell, :reevaluate_automatically, false)
info.eval.status == :ready and info.eval.validity == :stale and
Map.get(cell, :reevaluate_automatically, false)
end) end)
data_actions data_actions

View file

@ -819,8 +819,8 @@ defmodule LivebookWeb.SessionLive do
with {:ok, cell, section} <- Notebook.fetch_cell_and_section(data.notebook, cell_id) do with {:ok, cell, section} <- Notebook.fetch_cell_and_section(data.notebook, cell_id) do
if data.runtime do if data.runtime do
ref = make_ref() ref = make_ref()
prev_locator = Session.find_prev_locator(data.notebook, cell, section) base_locator = Session.find_base_locator(data, cell, section, existing: true)
Runtime.handle_intellisense(data.runtime, self(), ref, request, prev_locator) Runtime.handle_intellisense(data.runtime, self(), ref, request, base_locator)
{:reply, %{"ref" => inspect(ref)}, socket} {:reply, %{"ref" => inspect(ref)}, socket}
else else

View file

@ -128,7 +128,7 @@ defmodule LivebookWeb.SessionLive.CellComponent do
<div class="relative"> <div class="relative">
<div data-element="ui-box"> <div data-element="ui-box">
<%= case @cell_view.status do %> <%= case @cell_view.status do %>
<% :alive -> %> <% :started -> %>
<.live_component module={LivebookWeb.JSViewComponent} <.live_component module={LivebookWeb.JSViewComponent}
id={@cell_view.id} id={@cell_view.id}
js_view={@cell_view.js_view} js_view={@cell_view.js_view}

View file

@ -3,11 +3,11 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServerTest do
alias Livebook.Runtime.ErlDist.{NodeManager, RuntimeServer} alias Livebook.Runtime.ErlDist.{NodeManager, RuntimeServer}
setup do setup ctx do
{:ok, manager_pid} = {:ok, manager_pid} =
start_supervised({NodeManager, [unload_modules_on_termination: false, anonymous: true]}) start_supervised({NodeManager, [unload_modules_on_termination: false, anonymous: true]})
runtime_server_pid = NodeManager.start_runtime_server(manager_pid) runtime_server_pid = NodeManager.start_runtime_server(manager_pid, ctx[:opts] || [])
RuntimeServer.attach(runtime_server_pid, self()) RuntimeServer.attach(runtime_server_pid, self())
{:ok, %{pid: runtime_server_pid, manager_pid: manager_pid}} {:ok, %{pid: runtime_server_pid, manager_pid: manager_pid}}
end end
@ -204,4 +204,71 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServerTest do
assert_receive {:runtime_container_down, :c1, message} assert_receive {:runtime_container_down, :c1, message}
assert message =~ "killed" assert message =~ "killed"
end end
describe "smart cells" do
defmodule Kino.SmartCell.Dumb do
use GenServer
# Every smart cell needs a child_spec, we use the GenServer default
def start_link(info) do
{:ok, pid} = GenServer.start_link(__MODULE__, info)
{:ok, pid,
%{
js_view: %{ref: info.ref, pid: pid, assets: %{}},
source: "source",
scan_binding: fn pid, _binding, _env -> send(pid, :scan_binding_result) end
}}
end
@impl true
def init(info) do
{:ok, info}
end
@impl true
def handle_info(message, info) do
send(info.target_pid, {:smart_cell_debug, info.ref, :handle_info, message})
{:noreply, info}
end
end
defmodule Kino.SmartCell do
def definitions() do
[%{kind: "dumb", module: Kino.SmartCell.Dumb, name: "Test smart cell"}]
end
end
@opts [smart_cell_definitions_module: Kino.SmartCell]
@tag opts: @opts
test "notifies runtime owner when a smart cell is started", %{pid: pid} do
RuntimeServer.start_smart_cell(pid, "dumb", "ref", %{}, {:c1, nil})
assert_receive {:runtime_smart_cell_started, "ref", %{js_view: %{}, source: "source"}}
end
@tag opts: @opts
test "once started scans binding and sends the result to the cell server", %{pid: pid} do
RuntimeServer.start_smart_cell(pid, "dumb", "ref", %{}, {:c1, nil})
assert_receive {:smart_cell_debug, "ref", :handle_info, :scan_binding_result}
end
@tag opts: @opts
test "scans binding when a new base locator is set", %{pid: pid} do
RuntimeServer.start_smart_cell(pid, "dumb", "ref", %{}, {:c1, nil})
assert_receive {:smart_cell_debug, "ref", :handle_info, :scan_binding_result}
RuntimeServer.set_smart_cell_base_locator(pid, "ref", {:c2, nil})
assert_receive {:smart_cell_debug, "ref", :handle_info, :scan_binding_result}
end
@tag opts: @opts
test "scans binding when the base locator is evaluated", %{pid: pid} do
RuntimeServer.evaluate_code(pid, "1 + 1", {:c1, :e1}, {:c1, nil})
RuntimeServer.start_smart_cell(pid, "dumb", "ref", %{}, {:c1, :e1})
assert_receive {:smart_cell_debug, "ref", :handle_info, :scan_binding_result}
RuntimeServer.evaluate_code(pid, "1 + 1", {:c1, :e1}, {:c1, nil})
assert_receive {:smart_cell_debug, "ref", :handle_info, :scan_binding_result}
end
end
end end

View file

@ -35,7 +35,7 @@ defmodule Livebook.Runtime.EvaluatorTest do
metadata.memory_usage metadata.memory_usage
end end
test "given no prev_ref does not see previous evaluation context", %{evaluator: evaluator} do test "given no base_ref does not see previous evaluation context", %{evaluator: evaluator} do
Evaluator.evaluate_code(evaluator, "x = 1", :code_1) Evaluator.evaluate_code(evaluator, "x = 1", :code_1)
assert_receive {:runtime_evaluation_response, :code_1, _, metadata()} assert_receive {:runtime_evaluation_response, :code_1, _, metadata()}
@ -50,7 +50,7 @@ defmodule Livebook.Runtime.EvaluatorTest do
end) end)
end end
test "given prev_ref sees previous evaluation context", %{evaluator: evaluator} do test "given base_ref sees previous evaluation context", %{evaluator: evaluator} do
Evaluator.evaluate_code(evaluator, "x = 1", :code_1) Evaluator.evaluate_code(evaluator, "x = 1", :code_1)
assert_receive {:runtime_evaluation_response, :code_1, _, metadata()} assert_receive {:runtime_evaluation_response, :code_1, _, metadata()}
@ -59,7 +59,7 @@ defmodule Livebook.Runtime.EvaluatorTest do
assert_receive {:runtime_evaluation_response, :code_2, {:ok, 1}, metadata()} assert_receive {:runtime_evaluation_response, :code_2, {:ok, 1}, metadata()}
end end
test "given invalid prev_ref just uses default context", %{evaluator: evaluator} do test "given invalid base_ref just uses default context", %{evaluator: evaluator} do
Evaluator.evaluate_code(evaluator, ":hey", :code_1, :code_nonexistent) Evaluator.evaluate_code(evaluator, ":hey", :code_1, :code_nonexistent)
assert_receive {:runtime_evaluation_response, :code_1, {:ok, :hey}, metadata()} assert_receive {:runtime_evaluation_response, :code_1, {:ok, :hey}, metadata()}

View file

@ -1,6 +1,8 @@
defmodule Livebook.Session.DataTest do defmodule Livebook.Session.DataTest do
use ExUnit.Case, async: true use ExUnit.Case, async: true
import Livebook.TestHelpers
alias Livebook.Session.Data alias Livebook.Session.Data
alias Livebook.{Delta, Notebook} alias Livebook.{Delta, Notebook}
alias Livebook.Notebook.Cell alias Livebook.Notebook.Cell
@ -374,7 +376,7 @@ defmodule Livebook.Session.DataTest do
cell_infos: %{ cell_infos: %{
"c1" => %{eval: %{validity: :evaluated}}, "c1" => %{eval: %{validity: :evaluated}},
"c2" => %{eval: %{validity: :aborted, status: :ready}}, "c2" => %{eval: %{validity: :aborted, status: :ready}},
"c3" => %{eval: %{validity: :stale, status: :evaluating}} "c3" => %{eval: %{validity: :fresh, status: :evaluating}}
} }
}, _actions} = Data.apply_operation(data, operation) }, _actions} = Data.apply_operation(data, operation)
end end
@ -447,7 +449,24 @@ defmodule Livebook.Session.DataTest do
operation = {:insert_cell, self(), "s1", 0, :smart, "c1", %{kind: "text"}} operation = {:insert_cell, self(), "s1", 0, :smart, "c1", %{kind: "text"}}
assert {:ok, %{cell_infos: %{"c1" => %{status: :starting}}}, assert {:ok, %{cell_infos: %{"c1" => %{status: :starting}}},
[{:start_smart_cell, %{id: "c1"}}]} = Data.apply_operation(data, operation) [{:start_smart_cell, %{id: "c1"}, %{id: "s1"}}]} =
Data.apply_operation(data, operation)
end
test "inserting code cell before smart cell does not change its base" do
data =
data_after_operations!([
{:insert_section, self(), 0, "s1"},
{:insert_cell, self(), "s1", 0, :code, "c1", %{}},
{:insert_cell, self(), "s1", 1, :smart, "c2", %{kind: "text"}},
{:set_runtime, self(), NoopRuntime.new()},
{:set_smart_cell_definitions, self(), [%{kind: "text", name: "Text"}]},
{:smart_cell_started, self(), "c2", Delta.new(), %{}}
])
operation = {:insert_cell, self(), "s1", 0, :code, "c3", %{}}
assert {:ok, %{}, []} = Data.apply_operation(data, operation)
end end
end end
@ -802,6 +821,28 @@ defmodule Livebook.Session.DataTest do
assert {:ok, _data, [{:stop_smart_cell, %{id: "c1"}}]} = assert {:ok, _data, [{:stop_smart_cell, %{id: "c1"}}]} =
Data.apply_operation(data, operation) Data.apply_operation(data, operation)
end end
test "deleting evaluated code cell before smart cell changes its base" do
data =
data_after_operations!([
{:insert_section, self(), 0, "s1"},
{:insert_cell, self(), "s1", 0, :code, "c1", %{}},
{:insert_cell, self(), "s1", 1, :smart, "c2", %{kind: "text"}},
{:set_runtime, self(), NoopRuntime.new()},
{:set_smart_cell_definitions, self(), [%{kind: "text", name: "Text"}]},
{:smart_cell_started, self(), "c2", Delta.new(), %{}},
{:queue_cells_evaluation, self(), ["c1"]},
{:add_cell_evaluation_response, self(), "c1", @eval_resp, @eval_meta}
])
operation = {:delete_cell, self(), "c1"}
assert {:ok, %{},
[
{:forget_evaluation, _, _},
{:set_smart_cell_base, %{id: "c2"}, %{id: "s1"}, nil}
]} = Data.apply_operation(data, operation)
end
end end
describe "apply_operation/2 given :restore_cell" do describe "apply_operation/2 given :restore_cell" do
@ -885,7 +926,8 @@ defmodule Livebook.Session.DataTest do
operation = {:restore_cell, self(), "c1"} operation = {:restore_cell, self(), "c1"}
assert {:ok, %{cell_infos: %{"c1" => %{status: :starting}}}, assert {:ok, %{cell_infos: %{"c1" => %{status: :starting}}},
[{:start_smart_cell, %{id: "c1"}}]} = Data.apply_operation(data, operation) [{:start_smart_cell, %{id: "c1"}, %{id: "s1"}}]} =
Data.apply_operation(data, operation)
end end
end end
@ -1685,7 +1727,7 @@ defmodule Livebook.Session.DataTest do
assert {:ok, assert {:ok,
%{ %{
cell_infos: %{ cell_infos: %{
"c1" => %{eval: %{validity: :evaluated, status: :evaluating}}, "c1" => %{eval: %{validity: :fresh, status: :evaluating}},
"c2" => %{eval: %{validity: :fresh, status: :ready}}, "c2" => %{eval: %{validity: :fresh, status: :ready}},
"c3" => %{eval: %{validity: :fresh, status: :queued}} "c3" => %{eval: %{validity: :fresh, status: :queued}}
}, },
@ -2304,6 +2346,25 @@ defmodule Livebook.Session.DataTest do
assert {:ok, %{input_values: %{"i1" => "value"}}, _} = Data.apply_operation(data, operation) assert {:ok, %{input_values: %{"i1" => "value"}}, _} = Data.apply_operation(data, operation)
end end
test "evaluating code cell before smart cell changes its base" do
data =
data_after_operations!([
{:insert_section, self(), 0, "s1"},
{:insert_cell, self(), "s1", 0, :code, "c1", %{}},
{:insert_cell, self(), "s1", 1, :smart, "c2", %{kind: "text"}},
{:set_runtime, self(), NoopRuntime.new()},
{:set_smart_cell_definitions, self(), [%{kind: "text", name: "Text"}]},
{:smart_cell_started, self(), "c2", Delta.new(), %{}},
{:queue_cells_evaluation, self(), ["c1"]}
])
operation = {:add_cell_evaluation_response, self(), "c1", @eval_resp, @eval_meta}
assert {:ok, %{},
[{:set_smart_cell_base, %{id: "c2"}, %{id: "s1"}, {%{id: "c1"}, %{id: "s1"}}}]} =
Data.apply_operation(data, operation)
end
end end
describe "apply_operation/2 given :bind_input" do describe "apply_operation/2 given :bind_input" do
@ -2407,7 +2468,7 @@ defmodule Livebook.Session.DataTest do
cell_infos: %{ cell_infos: %{
"c1" => %{eval: %{validity: :aborted, status: :ready}}, "c1" => %{eval: %{validity: :aborted, status: :ready}},
"c2" => %{eval: %{validity: :evaluated, status: :ready}}, "c2" => %{eval: %{validity: :evaluated, status: :ready}},
"c3" => %{eval: %{validity: :evaluated, status: :evaluating}} "c3" => %{eval: %{validity: :fresh, status: :evaluating}}
}, },
section_infos: %{ section_infos: %{
"s1" => %{evaluating_cell_id: nil, evaluation_queue: []}, "s1" => %{evaluating_cell_id: nil, evaluation_queue: []},
@ -2443,7 +2504,7 @@ defmodule Livebook.Session.DataTest do
"c1" => %{eval: %{validity: :evaluated, status: :ready}}, "c1" => %{eval: %{validity: :evaluated, status: :ready}},
"c2" => %{eval: %{validity: :aborted, status: :ready}}, "c2" => %{eval: %{validity: :aborted, status: :ready}},
"c3" => %{eval: %{validity: :aborted, status: :ready}}, "c3" => %{eval: %{validity: :aborted, status: :ready}},
"c4" => %{eval: %{validity: :evaluated, status: :evaluating}} "c4" => %{eval: %{validity: :fresh, status: :evaluating}}
}, },
section_infos: %{ section_infos: %{
"s1" => %{evaluating_cell_id: nil, evaluation_queue: []}, "s1" => %{evaluating_cell_id: nil, evaluation_queue: []},
@ -2544,7 +2605,7 @@ defmodule Livebook.Session.DataTest do
"c1" => %{eval: %{validity: :evaluated, status: :ready}}, "c1" => %{eval: %{validity: :evaluated, status: :ready}},
"c2" => %{eval: %{validity: :aborted, status: :ready}}, "c2" => %{eval: %{validity: :aborted, status: :ready}},
"c3" => %{eval: %{validity: :fresh, status: :ready}}, "c3" => %{eval: %{validity: :fresh, status: :ready}},
"c4" => %{eval: %{validity: :evaluated, status: :evaluating}} "c4" => %{eval: %{validity: :fresh, status: :evaluating}}
}, },
section_infos: %{ section_infos: %{
"s1" => %{evaluating_cell_id: nil, evaluation_queue: []}, "s1" => %{evaluating_cell_id: nil, evaluation_queue: []},
@ -2622,7 +2683,7 @@ defmodule Livebook.Session.DataTest do
operation = {:smart_cell_started, self(), "c1", delta, %{}} operation = {:smart_cell_started, self(), "c1", delta, %{}}
assert {:ok, %{cell_infos: %{"c1" => %{status: :alive}}}, _actions} = assert {:ok, %{cell_infos: %{"c1" => %{status: :started}}}, _actions} =
Data.apply_operation(data, operation) Data.apply_operation(data, operation)
end end
@ -3381,18 +3442,6 @@ defmodule Livebook.Session.DataTest do
end end
end end
defp data_after_operations!(operations) do
Enum.reduce(operations, Data.new(), fn operation, data ->
case Data.apply_operation(data, operation) do
{:ok, data, _action} ->
data
:error ->
raise "failed to set up test data, operation #{inspect(operation)} returned an error"
end
end)
end
describe "bound_cells_with_section/2" do describe "bound_cells_with_section/2" do
test "returns an empty list when an invalid input id is given" do test "returns an empty list when an invalid input id is given" do
data = Data.new() data = Data.new()

View file

@ -1,8 +1,11 @@
defmodule Livebook.SessionTest do defmodule Livebook.SessionTest do
use ExUnit.Case, async: true use ExUnit.Case, async: true
import Livebook.TestHelpers
alias Livebook.{Session, Delta, Runtime, Utils, Notebook, FileSystem} alias Livebook.{Session, Delta, Runtime, Utils, Notebook, FileSystem}
alias Livebook.Notebook.{Section, Cell} alias Livebook.Notebook.{Section, Cell}
alias Livebook.Session.Data
setup do setup do
session = start_session() session = start_session()
@ -587,7 +590,7 @@ defmodule Livebook.SessionTest do
end end
end end
describe "find_prev_locator/3" do describe "find_base_locator/3" do
test "given cell in main flow returns previous Code cell" do test "given cell in main flow returns previous Code cell" do
cell1 = %{Cell.new(:code) | id: "c1"} cell1 = %{Cell.new(:code) | id: "c1"}
cell2 = %{Cell.new(:markdown) | id: "c2"} cell2 = %{Cell.new(:markdown) | id: "c2"}
@ -597,8 +600,9 @@ defmodule Livebook.SessionTest do
section2 = %{Section.new() | id: "s2", cells: [cell3]} section2 = %{Section.new() | id: "s2", cells: [cell3]}
notebook = %{Notebook.new() | sections: [section1, section2]} notebook = %{Notebook.new() | sections: [section1, section2]}
data = Data.new(notebook)
assert {:main_flow, "c1"} = Session.find_prev_locator(notebook, cell3, section2) assert {:main_flow, "c1"} = Session.find_base_locator(data, cell3, section2)
end end
test "given cell in branching section returns previous Code cell in that section" do test "given cell in branching section returns previous Code cell in that section" do
@ -616,8 +620,9 @@ defmodule Livebook.SessionTest do
} }
notebook = %{Notebook.new() | sections: [section1, section2]} notebook = %{Notebook.new() | sections: [section1, section2]}
data = Data.new(notebook)
assert {"s2", "c1"} = Session.find_prev_locator(notebook, cell3, section2) assert {"s2", "c1"} = Session.find_base_locator(data, cell3, section2)
end end
test "given cell in main flow returns nil if there is no previous cell" do test "given cell in main flow returns nil if there is no previous cell" do
@ -628,8 +633,9 @@ defmodule Livebook.SessionTest do
section2 = %{Section.new() | id: "s2", cells: [cell2]} section2 = %{Section.new() | id: "s2", cells: [cell2]}
notebook = %{Notebook.new() | sections: [section1, section2]} notebook = %{Notebook.new() | sections: [section1, section2]}
data = Data.new(notebook)
assert {:main_flow, nil} = Session.find_prev_locator(notebook, cell2, section2) assert {:main_flow, nil} = Session.find_base_locator(data, cell2, section2)
end end
test "given cell in branching section returns nil in that section if there is no previous cell" do test "given cell in branching section returns nil in that section if there is no previous cell" do
@ -646,8 +652,39 @@ defmodule Livebook.SessionTest do
} }
notebook = %{Notebook.new() | sections: [section1, section2]} notebook = %{Notebook.new() | sections: [section1, section2]}
data = Data.new(notebook)
assert {"s2", nil} = Session.find_prev_locator(notebook, cell2, section2) assert {"s2", nil} = Session.find_base_locator(data, cell2, section2)
end
test "when :existing is set ignores fresh and aborted cells" do
cell1 = %{Cell.new(:code) | id: "c1"}
cell2 = %{Cell.new(:code) | id: "c2"}
section1 = %{Section.new() | id: "s1", cells: [cell1, cell2]}
cell3 = %{Cell.new(:code) | id: "c3"}
section2 = %{Section.new() | id: "s2", cells: [cell3]}
notebook = %{Notebook.new() | sections: [section1, section2]}
data = Data.new(notebook)
assert {:main_flow, nil} = Session.find_base_locator(data, cell3, section2, existing: true)
data =
data_after_operations!(data, [
{:set_runtime, self(), Livebook.Runtime.NoopRuntime.new()},
{:queue_cells_evaluation, self(), ["c1"]},
{:add_cell_evaluation_response, self(), "c1", {:ok, nil}, %{evaluation_time_ms: 10}}
])
assert {:main_flow, "c1"} = Session.find_base_locator(data, cell3, section2, existing: true)
data =
data_after_operations!(data, [
{:reflect_main_evaluation_failure, self()}
])
assert {:main_flow, nil} = Session.find_base_locator(data, cell3, section2, existing: true)
end end
end end

View file

@ -25,8 +25,8 @@ defmodule Livebook.Runtime.NoopRuntime do
end end
end end
def start_smart_cell(_, _, _, _), do: :ok def start_smart_cell(_, _, _, _, _), do: :ok
def set_smart_cell_base_locator(_, _, _), do: :ok
def stop_smart_cell(_, _), do: :ok def stop_smart_cell(_, _), do: :ok
end end
end end

View file

@ -1,6 +1,8 @@
defmodule Livebook.TestHelpers do defmodule Livebook.TestHelpers do
@moduledoc false @moduledoc false
alias Livebook.Session.Data
@doc """ @doc """
Creates file structure according to the given specification. Creates file structure according to the given specification.
""" """
@ -18,4 +20,21 @@ defmodule Livebook.TestHelpers do
end end
end end
end end
@doc """
Applies the given list of operations to `Livebook.Session.Data`.
Raises if any of the operations results in an error.
"""
def data_after_operations!(data \\ Data.new(), operations) do
Enum.reduce(operations, data, fn operation, data ->
case Data.apply_operation(data, operation) do
{:ok, data, _action} ->
data
:error ->
raise "failed to set up test data, operation #{inspect(operation)} returned an error"
end
end)
end
end end