From 6b7825871338af0ec3f4196ec3e17d2670e6a92c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jonatan=20K=C5=82osko?= Date: Sat, 5 Mar 2022 13:19:42 +0100 Subject: [PATCH] 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 --- lib/livebook/runtime.ex | 34 ++- lib/livebook/runtime/attached.ex | 22 +- lib/livebook/runtime/elixir_standalone.ex | 22 +- lib/livebook/runtime/embedded.ex | 22 +- lib/livebook/runtime/erl_dist/node_manager.ex | 10 +- .../runtime/erl_dist/runtime_server.ex | 223 ++++++++++++++---- lib/livebook/runtime/evaluator.ex | 31 ++- lib/livebook/runtime/mix_standalone.ex | 22 +- lib/livebook/session.ex | 59 ++++- lib/livebook/session/data.ex | 102 +++++++- lib/livebook_web/live/session_live.ex | 4 +- .../live/session_live/cell_component.ex | 2 +- .../runtime/erl_dist/runtime_server_test.exs | 71 +++++- test/livebook/runtime/evaluator_test.exs | 6 +- test/livebook/session/data_test.exs | 89 +++++-- test/livebook/session_test.exs | 47 +++- test/support/noop_runtime.ex | 4 +- test/support/test_helpers.ex | 19 ++ 18 files changed, 648 insertions(+), 141 deletions(-) diff --git a/lib/livebook/runtime.ex b/lib/livebook/runtime.ex index 7092d5b8f..3c5889f4c 100644 --- a/lib/livebook/runtime.ex +++ b/lib/livebook/runtime.ex @@ -208,6 +208,10 @@ defprotocol Livebook.Runtime do } } + @type smart_cell_ref :: String.t() + + @type smart_cell_attrs :: map() + @doc """ 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 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, the previous evaluation reference may be specified as `nil`. @@ -284,7 +288,7 @@ defprotocol Livebook.Runtime do the value of `__DIR__` """ @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 """ Disposes of an evaluation identified by the given locator. @@ -314,11 +318,11 @@ defprotocol Livebook.Runtime do * `{:runtime_intellisense_response, ref, request, response}`. - The given `locator` idenfities an evaluation that may be used - as the context when resolving the request (if relevant). + The given `base_locator` idenfities an evaluation that may be + used as the context when resolving the request (if relevant). """ @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 """ 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 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 * `{: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 deserialized as JSON. """ - @spec start_smart_cell(t(), String.t(), String.t(), term()) :: :ok - def start_smart_cell(runtime, kind, ref, attrs) + @spec start_smart_cell(t(), String.t(), smart_cell_ref(), smart_cell_attrs(), locator()) :: :ok + 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 """ 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) end diff --git a/lib/livebook/runtime/attached.ex b/lib/livebook/runtime/attached.ex index d684c4e89..f7ce1da36 100644 --- a/lib/livebook/runtime/attached.ex +++ b/lib/livebook/runtime/attached.ex @@ -50,8 +50,8 @@ defimpl Livebook.Runtime, for: Livebook.Runtime.Attached do ErlDist.RuntimeServer.stop(runtime.server_pid) end - def evaluate_code(runtime, code, locator, prev_locator, opts \\ []) do - ErlDist.RuntimeServer.evaluate_code(runtime.server_pid, code, locator, prev_locator, opts) + def evaluate_code(runtime, code, locator, base_locator, opts \\ []) do + ErlDist.RuntimeServer.evaluate_code(runtime.server_pid, code, locator, base_locator, opts) end 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) end - def handle_intellisense(runtime, send_to, ref, request, locator) do - ErlDist.RuntimeServer.handle_intellisense(runtime.server_pid, send_to, ref, request, locator) + def handle_intellisense(runtime, send_to, ref, request, base_locator) do + ErlDist.RuntimeServer.handle_intellisense( + runtime.server_pid, + send_to, + ref, + request, + base_locator + ) end def duplicate(runtime) do @@ -79,8 +85,12 @@ defimpl Livebook.Runtime, for: Livebook.Runtime.Attached do ErlDist.RuntimeServer.read_file(runtime.server_pid, path) end - def start_smart_cell(runtime, kind, ref, attrs) do - ErlDist.RuntimeServer.start_smart_cell(runtime.server_pid, kind, ref, attrs) + def start_smart_cell(runtime, kind, ref, attrs, base_locator) do + 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 def stop_smart_cell(runtime, ref) do diff --git a/lib/livebook/runtime/elixir_standalone.ex b/lib/livebook/runtime/elixir_standalone.ex index f343f7dd3..d18b7ad7c 100644 --- a/lib/livebook/runtime/elixir_standalone.ex +++ b/lib/livebook/runtime/elixir_standalone.ex @@ -78,8 +78,8 @@ defimpl Livebook.Runtime, for: Livebook.Runtime.ElixirStandalone do ErlDist.RuntimeServer.stop(runtime.server_pid) end - def evaluate_code(runtime, code, locator, prev_locator, opts \\ []) do - ErlDist.RuntimeServer.evaluate_code(runtime.server_pid, code, locator, prev_locator, opts) + def evaluate_code(runtime, code, locator, base_locator, opts \\ []) do + ErlDist.RuntimeServer.evaluate_code(runtime.server_pid, code, locator, base_locator, opts) end 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) end - def handle_intellisense(runtime, send_to, ref, request, locator) do - ErlDist.RuntimeServer.handle_intellisense(runtime.server_pid, send_to, ref, request, locator) + def handle_intellisense(runtime, send_to, ref, request, base_locator) do + ErlDist.RuntimeServer.handle_intellisense( + runtime.server_pid, + send_to, + ref, + request, + base_locator + ) end def duplicate(_runtime) do @@ -104,8 +110,12 @@ defimpl Livebook.Runtime, for: Livebook.Runtime.ElixirStandalone do ErlDist.RuntimeServer.read_file(runtime.server_pid, path) end - def start_smart_cell(runtime, kind, ref, attrs) do - ErlDist.RuntimeServer.start_smart_cell(runtime.server_pid, kind, ref, attrs) + def start_smart_cell(runtime, kind, ref, attrs, base_locator) do + 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 def stop_smart_cell(runtime, ref) do diff --git a/lib/livebook/runtime/embedded.ex b/lib/livebook/runtime/embedded.ex index 5912fdef0..fd90ad398 100644 --- a/lib/livebook/runtime/embedded.ex +++ b/lib/livebook/runtime/embedded.ex @@ -51,8 +51,8 @@ defimpl Livebook.Runtime, for: Livebook.Runtime.Embedded do ErlDist.RuntimeServer.stop(runtime.server_pid) end - def evaluate_code(runtime, code, locator, prev_locator, opts \\ []) do - ErlDist.RuntimeServer.evaluate_code(runtime.server_pid, code, locator, prev_locator, opts) + def evaluate_code(runtime, code, locator, base_locator, opts \\ []) do + ErlDist.RuntimeServer.evaluate_code(runtime.server_pid, code, locator, base_locator, opts) end 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) end - def handle_intellisense(runtime, send_to, ref, request, locator) do - ErlDist.RuntimeServer.handle_intellisense(runtime.server_pid, send_to, ref, request, locator) + def handle_intellisense(runtime, send_to, ref, request, base_locator) do + ErlDist.RuntimeServer.handle_intellisense( + runtime.server_pid, + send_to, + ref, + request, + base_locator + ) end def duplicate(_runtime) do @@ -77,8 +83,12 @@ defimpl Livebook.Runtime, for: Livebook.Runtime.Embedded do ErlDist.RuntimeServer.read_file(runtime.server_pid, path) end - def start_smart_cell(runtime, kind, ref, attrs) do - ErlDist.RuntimeServer.start_smart_cell(runtime.server_pid, kind, ref, attrs) + def start_smart_cell(runtime, kind, ref, attrs, base_locator) do + 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 def stop_smart_cell(runtime, ref) do diff --git a/lib/livebook/runtime/erl_dist/node_manager.ex b/lib/livebook/runtime/erl_dist/node_manager.ex index ad2ab3eed..38685725d 100644 --- a/lib/livebook/runtime/erl_dist/node_manager.ex +++ b/lib/livebook/runtime/erl_dist/node_manager.ex @@ -71,9 +71,9 @@ defmodule Livebook.Runtime.ErlDist.NodeManager do @doc """ Starts a new `Livebook.Runtime.ErlDist.RuntimeServer` for evaluation. """ - @spec start_runtime_server(node() | pid()) :: pid() - def start_runtime_server(node_or_pid) do - GenServer.call(server(node_or_pid), :start_runtime_server) + @spec start_runtime_server(node() | pid(), keyword()) :: pid() + def start_runtime_server(node_or_pid, opts \\ []) do + GenServer.call(server(node_or_pid), {:start_runtime_server, opts}) end 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} @impl true - def handle_call(:start_runtime_server, _from, state) do + def handle_call({:start_runtime_server, opts}, _from, state) do {:ok, server_pid} = - DynamicSupervisor.start_child(state.server_supevisor, ErlDist.RuntimeServer) + DynamicSupervisor.start_child(state.server_supevisor, {ErlDist.RuntimeServer, opts}) Process.monitor(server_pid) state = update_in(state.runtime_servers, &[server_pid | &1]) diff --git a/lib/livebook/runtime/erl_dist/runtime_server.ex b/lib/livebook/runtime/erl_dist/runtime_server.ex index 9b41b99f3..e691f2ec1 100644 --- a/lib/livebook/runtime/erl_dist/runtime_server.ex +++ b/lib/livebook/runtime/erl_dist/runtime_server.ex @@ -15,6 +15,8 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServer do use GenServer, restart: :temporary + require Logger + alias Livebook.Runtime.Evaluator alias Livebook.Runtime 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 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 GenServer.start_link(__MODULE__, opts) @@ -60,8 +68,8 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServer do See `Livebook.Runtime.Evaluator` for more details. """ @spec evaluate_code(pid(), String.t(), Runtime.locator(), Runtime.locator(), keyword()) :: :ok - def evaluate_code(pid, code, locator, prev_locator, opts \\ []) do - GenServer.cast(pid, {:evaluate_code, code, locator, prev_locator, opts}) + def evaluate_code(pid, code, locator, base_locator, opts \\ []) do + GenServer.cast(pid, {:evaluate_code, code, locator, base_locator, opts}) end @doc """ @@ -100,8 +108,8 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServer do Runtime.intellisense_request(), Runtime.locator() ) :: :ok - def handle_intellisense(pid, send_to, ref, request, locator) do - GenServer.cast(pid, {:handle_intellisense, send_to, ref, request, locator}) + def handle_intellisense(pid, send_to, ref, request, base_locator) do + GenServer.cast(pid, {:handle_intellisense, send_to, ref, request, base_locator}) end @doc """ @@ -126,9 +134,23 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServer do @doc """ Starts a new smart cell. """ - @spec start_smart_cell(pid(), String.t(), String.t(), term()) :: :ok - def start_smart_cell(pid, kind, ref, attrs) do - GenServer.cast(pid, {:start_smart_cell, kind, ref, attrs}) + @spec start_smart_cell( + pid(), + 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 @doc """ @@ -151,7 +173,7 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServer do end @impl true - def init(_opts) do + def init(opts) do Process.send_after(self(), :check_owner, @await_owner_timeout) schedule_memory_usage_report() @@ -171,6 +193,8 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServer do smart_cell_gl: nil, smart_cells: %{}, smart_cell_definitions: [], + smart_cell_definitions_module: + Keyword.get(opts, :smart_cell_definitions_module, Kino.SmartCell), memory_timer_ref: nil }} end @@ -190,24 +214,18 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServer do {:stop, :shutdown, state} end - def handle_info({:DOWN, _, :process, pid, reason}, state) 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}) - {:noreply, %{state | evaluators: Map.delete(state.evaluators, container_ref)}} - - nil -> - {:noreply, state} - end + def handle_info({:DOWN, _, :process, _, _} = message, state) do + {:noreply, + state + |> handle_down_evaluator(message) + |> handle_down_scan_binding(message)} end - def handle_info({:evaluation_finished, _ref}, state) do - {:noreply, report_smart_cell_definitions(state)} + def handle_info({:evaluation_finished, pid, evaluation_ref}, state) do + {:noreply, + state + |> report_smart_cell_definitions() + |> scan_binding_after_evaluation(pid, evaluation_ref)} end def handle_info(:memory_usage, state) do @@ -216,8 +234,37 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServer do {:noreply, state} 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} + 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 def handle_cast({:attach, owner, opts}, state) do if state.owner do @@ -239,13 +286,13 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServer do end def handle_cast( - {:evaluate_code, code, {container_ref, evaluation_ref}, prev_locator, opts}, + {:evaluate_code, code, {container_ref, evaluation_ref}, base_locator, opts}, state ) do state = ensure_evaluator(state, container_ref) - prev_evaluation_ref = - case prev_locator do + base_evaluation_ref = + case base_locator do {^container_ref, evaluation_ref} -> evaluation_ref @@ -265,7 +312,7 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServer do state.evaluators[container_ref], code, evaluation_ref, - prev_evaluation_ref, + base_evaluation_ref, opts ) @@ -285,8 +332,8 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServer do {:noreply, state} end - def handle_cast({:handle_intellisense, send_to, ref, request, locator}, state) do - {container_ref, evaluation_ref} = locator + def handle_cast({:handle_intellisense, send_to, ref, request, base_locator}, state) do + {container_ref, evaluation_ref} = base_locator evaluator = state.evaluators[container_ref] intellisense_context = @@ -304,7 +351,7 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServer do {:noreply, state} 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)) state = @@ -313,8 +360,23 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServer do {definition.module, %{ref: ref, attrs: attrs, target_pid: state.owner}} ) do {:ok, pid, info} -> - send(state.owner, {:runtime_smart_cell_started, ref, info}) - put_in(state.smart_cells[ref], pid) + %{js_view: js_view, source: source, scan_binding: scan_binding} = info + + 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 @@ -323,8 +385,18 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServer do {:noreply, state} 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 - {pid, state} = pop_in(state.smart_cells[ref]) + {%{pid: pid}, state} = pop_in(state.smart_cells[ref]) if pid do DynamicSupervisor.terminate_child(state.smart_cell_supervisor, pid) @@ -393,7 +465,7 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServer do end 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 state @@ -404,13 +476,84 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServer do end end - @compile {:no_warn_undefined, {Kino.SmartCell, :definitions, 0}} - - defp get_smart_cell_definitions() do - if Code.ensure_loaded?(Kino.SmartCell) and function_exported?(Kino.SmartCell, :definitions, 0) do - Kino.SmartCell.definitions() + defp get_smart_cell_definitions(module) do + if Code.ensure_loaded?(module) and function_exported?(module, :definitions, 0) do + module.definitions() else [] 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 diff --git a/lib/livebook/runtime/evaluator.ex b/lib/livebook/runtime/evaluator.ex index d096aec03..6e41c4b48 100644 --- a/lib/livebook/runtime/evaluator.ex +++ b/lib/livebook/runtime/evaluator.ex @@ -118,7 +118,7 @@ defmodule Livebook.Runtime.Evaluator do response. 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 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 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 - def evaluate_code(evaluator, code, ref, prev_ref \\ nil, opts \\ []) when ref != nil do - cast(evaluator, {:evaluate_code, code, ref, prev_ref, opts}) + def evaluate_code(evaluator, code, ref, base_ref \\ nil, opts \\ []) when ref != nil do + cast(evaluator, {:evaluate_code, code, ref, base_ref, opts}) end @doc """ @@ -216,6 +216,17 @@ defmodule Livebook.Runtime.Evaluator do call(evaluator, {:map_binding, ref, fun}) 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 send(evaluator.pid, {:cast, evaluator.ref, message}) :ok @@ -298,12 +309,12 @@ defmodule Livebook.Runtime.Evaluator do %{binding: [], env: env, id: random_id()} 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.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") context = put_in(context.env.file, file) start_time = System.monotonic_time() @@ -338,7 +349,7 @@ defmodule Livebook.Runtime.Evaluator do send(state.send_to, {:runtime_evaluation_response, ref, output, metadata}) if notify_to = opts[:notify_to] do - send(notify_to, {:evaluation_finished, ref}) + send(notify_to, {:evaluation_finished, self(), ref}) end :erlang.garbage_collect(self()) @@ -353,6 +364,12 @@ defmodule Livebook.Runtime.Evaluator do {:noreply, state} 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 context = get_context(state, ref) diff --git a/lib/livebook/runtime/mix_standalone.ex b/lib/livebook/runtime/mix_standalone.ex index f6421d512..96593c34f 100644 --- a/lib/livebook/runtime/mix_standalone.ex +++ b/lib/livebook/runtime/mix_standalone.ex @@ -145,8 +145,8 @@ defimpl Livebook.Runtime, for: Livebook.Runtime.MixStandalone do ErlDist.RuntimeServer.stop(runtime.server_pid) end - def evaluate_code(runtime, code, locator, prev_locator, opts \\ []) do - ErlDist.RuntimeServer.evaluate_code(runtime.server_pid, code, locator, prev_locator, opts) + def evaluate_code(runtime, code, locator, base_locator, opts \\ []) do + ErlDist.RuntimeServer.evaluate_code(runtime.server_pid, code, locator, base_locator, opts) end 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) end - def handle_intellisense(runtime, send_to, ref, request, locator) do - ErlDist.RuntimeServer.handle_intellisense(runtime.server_pid, send_to, ref, request, locator) + def handle_intellisense(runtime, send_to, ref, request, base_locator) do + ErlDist.RuntimeServer.handle_intellisense( + runtime.server_pid, + send_to, + ref, + request, + base_locator + ) end def duplicate(runtime) do @@ -171,8 +177,12 @@ defimpl Livebook.Runtime, for: Livebook.Runtime.MixStandalone do ErlDist.RuntimeServer.read_file(runtime.server_pid, path) end - def start_smart_cell(runtime, kind, ref, attrs) do - ErlDist.RuntimeServer.start_smart_cell(runtime.server_pid, kind, ref, attrs) + def start_smart_cell(runtime, kind, ref, attrs, base_locator) do + 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 def stop_smart_cell(runtime, ref) do diff --git a/lib/livebook/session.ex b/lib/livebook/session.ex index 544d76593..558b34379 100644 --- a/lib/livebook/session.ex +++ b/lib/livebook/session.ex @@ -56,6 +56,7 @@ defmodule Livebook.Session do alias Livebook.Notebook.{Cell, Section} @timeout :infinity + @main_container_ref :main_flow @type t :: %__MODULE__{ id: id(), @@ -879,7 +880,7 @@ defmodule Livebook.Session do operation = 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} end @@ -1211,8 +1212,8 @@ defmodule Livebook.Session do opts = [file: file] locator = {container_ref_for_section(section), cell.id} - prev_locator = find_prev_locator(state.data.notebook, cell, section) - Runtime.evaluate_code(state.data.runtime, cell.source, locator, prev_locator, opts) + base_locator = find_base_locator(state.data, cell, section) + Runtime.evaluate_code(state.data.runtime, cell.source, locator, base_locator, opts) evaluation_digest = :erlang.md5(cell.source) handle_operation(state, {:evaluation_started, self(), cell.id, evaluation_digest}) @@ -1234,9 +1235,27 @@ defmodule Livebook.Session do state 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 - 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 state @@ -1461,20 +1480,36 @@ defmodule Livebook.Session do end @doc """ - Determines locator of the evaluation that the given - cell depends on. + Finds evaluation locator that the given 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() - def find_prev_locator(notebook, cell, section) do + @spec find_base_locator(Data.t(), Cell.t(), Section.t(), keyword()) :: Runtime.locator() + 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} - notebook + data.notebook |> Notebook.parent_cells_with_section(cell.id) |> 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 - 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 end diff --git a/lib/livebook/session/data.ex b/lib/livebook/session/data.ex index 0bbc8ff16..8f5be20d0 100644 --- a/lib/livebook/session/data.ex +++ b/lib/livebook/session/data.ex @@ -108,7 +108,7 @@ defmodule Livebook.Session.Data do @type cell_evaluation_validity :: :fresh | :evaluated | :stale | :aborted @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() @@ -186,7 +186,8 @@ defmodule Livebook.Session.Data do | {:start_evaluation, Cell.t(), Section.t()} | {:stop_evaluation, 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()} @doc """ @@ -358,6 +359,7 @@ defmodule Livebook.Session.Data do |> with_actions() |> delete_section(section, delete_cells) |> compute_snapshots_and_validity() + |> update_smart_cell_bases(data) |> set_dirty() |> wrap_ok() else @@ -371,6 +373,7 @@ defmodule Livebook.Session.Data do |> with_actions() |> delete_cell(cell, section) |> compute_snapshots_and_validity() + |> update_smart_cell_bases(data) |> set_dirty() |> wrap_ok() end @@ -398,6 +401,7 @@ defmodule Livebook.Session.Data do |> with_actions() |> move_cell(cell, offset) |> compute_snapshots_and_validity() + |> update_smart_cell_bases(data) |> set_dirty() |> wrap_ok() else @@ -413,6 +417,7 @@ defmodule Livebook.Session.Data do |> with_actions() |> move_section(section, offset) |> compute_snapshots_and_validity() + |> update_smart_cell_bases(data) |> set_dirty() |> wrap_ok() else @@ -482,6 +487,7 @@ defmodule Livebook.Session.Data do |> compute_snapshots_and_validity() |> maybe_evaluate_queued() |> compute_snapshots_and_validity() + |> update_smart_cell_bases(data) |> mark_dirty_if_persisting_outputs() |> wrap_ok() else @@ -507,6 +513,7 @@ defmodule Livebook.Session.Data do data |> with_actions() |> clear_main_evaluation() + |> update_smart_cell_bases(data) |> wrap_ok() end @@ -515,6 +522,7 @@ defmodule Livebook.Session.Data do data |> with_actions() |> clear_section_evaluation(section) + |> update_smart_cell_bases(data) |> wrap_ok() end end @@ -525,6 +533,7 @@ defmodule Livebook.Session.Data do data |> with_actions() |> cancel_cell_evaluation(cell, section) + |> update_smart_cell_bases(data) |> wrap_ok() else _ -> :error @@ -563,6 +572,7 @@ defmodule Livebook.Session.Data do |> with_actions() |> erase_outputs() |> garbage_collect_input_values() + |> update_smart_cell_bases(data) |> wrap_ok() end @@ -1203,7 +1213,7 @@ defmodule Livebook.Session.Data do data_actions |> 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}) end @@ -1367,26 +1377,26 @@ defmodule Livebook.Session.Data do defp maybe_start_smart_cells({data, _} = data_actions) 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) - 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 |> update_cell_info!(cell.id, &%{&1 | status: :starting}) - |> add_action({:start_smart_cell, cell}) + |> add_action({:start_smart_cell, cell, section}) end) else data_actions end end - defp dead_smart_cells(data) do + defp dead_smart_cells_with_section(data) do for section <- data.notebook.sections, %Cell.Smart{} = cell <- section.cells, info = data.cell_infos[cell.id], info.status == :dead, - do: cell + do: {cell, section} end defp clear_smart_cells({data, _} = data_actions) do @@ -1453,6 +1463,75 @@ defmodule Livebook.Session.Data do 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 %{ evaluating_cell_id: nil, @@ -1671,6 +1750,7 @@ defmodule Livebook.Session.Data do update_cell_eval_info!(data_actions, cell.id, fn eval_info -> validity = case eval_info do + %{status: :evaluating, validity: validity} -> validity %{evaluation_snapshot: snapshot, snapshot: snapshot} -> :evaluated %{evaluation_snapshot: nil, validity: :aborted} -> :aborted %{evaluation_snapshot: nil} -> :fresh @@ -1688,7 +1768,9 @@ defmodule Livebook.Session.Data do |> Notebook.evaluable_cells_with_section() |> Enum.filter(fn {cell, _section} -> 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) data_actions diff --git a/lib/livebook_web/live/session_live.ex b/lib/livebook_web/live/session_live.ex index 1bb1e86e8..693b13699 100644 --- a/lib/livebook_web/live/session_live.ex +++ b/lib/livebook_web/live/session_live.ex @@ -819,8 +819,8 @@ defmodule LivebookWeb.SessionLive do with {:ok, cell, section} <- Notebook.fetch_cell_and_section(data.notebook, cell_id) do if data.runtime do ref = make_ref() - prev_locator = Session.find_prev_locator(data.notebook, cell, section) - Runtime.handle_intellisense(data.runtime, self(), ref, request, prev_locator) + base_locator = Session.find_base_locator(data, cell, section, existing: true) + Runtime.handle_intellisense(data.runtime, self(), ref, request, base_locator) {:reply, %{"ref" => inspect(ref)}, socket} else diff --git a/lib/livebook_web/live/session_live/cell_component.ex b/lib/livebook_web/live/session_live/cell_component.ex index d0d17b854..c96eace6f 100644 --- a/lib/livebook_web/live/session_live/cell_component.ex +++ b/lib/livebook_web/live/session_live/cell_component.ex @@ -128,7 +128,7 @@ defmodule LivebookWeb.SessionLive.CellComponent do
<%= case @cell_view.status do %> - <% :alive -> %> + <% :started -> %> <.live_component module={LivebookWeb.JSViewComponent} id={@cell_view.id} js_view={@cell_view.js_view} diff --git a/test/livebook/runtime/erl_dist/runtime_server_test.exs b/test/livebook/runtime/erl_dist/runtime_server_test.exs index f36e08f5c..d9d2e0cbd 100644 --- a/test/livebook/runtime/erl_dist/runtime_server_test.exs +++ b/test/livebook/runtime/erl_dist/runtime_server_test.exs @@ -3,11 +3,11 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServerTest do alias Livebook.Runtime.ErlDist.{NodeManager, RuntimeServer} - setup do + setup ctx do {:ok, manager_pid} = 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()) {:ok, %{pid: runtime_server_pid, manager_pid: manager_pid}} end @@ -204,4 +204,71 @@ defmodule Livebook.Runtime.ErlDist.RuntimeServerTest do assert_receive {:runtime_container_down, :c1, message} assert message =~ "killed" 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 diff --git a/test/livebook/runtime/evaluator_test.exs b/test/livebook/runtime/evaluator_test.exs index f0a662f1e..30d170496 100644 --- a/test/livebook/runtime/evaluator_test.exs +++ b/test/livebook/runtime/evaluator_test.exs @@ -35,7 +35,7 @@ defmodule Livebook.Runtime.EvaluatorTest do metadata.memory_usage 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) assert_receive {:runtime_evaluation_response, :code_1, _, metadata()} @@ -50,7 +50,7 @@ defmodule Livebook.Runtime.EvaluatorTest do 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) 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()} 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) assert_receive {:runtime_evaluation_response, :code_1, {:ok, :hey}, metadata()} diff --git a/test/livebook/session/data_test.exs b/test/livebook/session/data_test.exs index 7bd97d99d..bb554c4f8 100644 --- a/test/livebook/session/data_test.exs +++ b/test/livebook/session/data_test.exs @@ -1,6 +1,8 @@ defmodule Livebook.Session.DataTest do use ExUnit.Case, async: true + import Livebook.TestHelpers + alias Livebook.Session.Data alias Livebook.{Delta, Notebook} alias Livebook.Notebook.Cell @@ -374,7 +376,7 @@ defmodule Livebook.Session.DataTest do cell_infos: %{ "c1" => %{eval: %{validity: :evaluated}}, "c2" => %{eval: %{validity: :aborted, status: :ready}}, - "c3" => %{eval: %{validity: :stale, status: :evaluating}} + "c3" => %{eval: %{validity: :fresh, status: :evaluating}} } }, _actions} = Data.apply_operation(data, operation) end @@ -447,7 +449,24 @@ defmodule Livebook.Session.DataTest do operation = {:insert_cell, self(), "s1", 0, :smart, "c1", %{kind: "text"}} 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 @@ -802,6 +821,28 @@ defmodule Livebook.Session.DataTest do assert {:ok, _data, [{:stop_smart_cell, %{id: "c1"}}]} = Data.apply_operation(data, operation) 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 describe "apply_operation/2 given :restore_cell" do @@ -885,7 +926,8 @@ defmodule Livebook.Session.DataTest do operation = {:restore_cell, self(), "c1"} 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 @@ -1685,7 +1727,7 @@ defmodule Livebook.Session.DataTest do assert {:ok, %{ cell_infos: %{ - "c1" => %{eval: %{validity: :evaluated, status: :evaluating}}, + "c1" => %{eval: %{validity: :fresh, status: :evaluating}}, "c2" => %{eval: %{validity: :fresh, status: :ready}}, "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) 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 describe "apply_operation/2 given :bind_input" do @@ -2407,7 +2468,7 @@ defmodule Livebook.Session.DataTest do cell_infos: %{ "c1" => %{eval: %{validity: :aborted, status: :ready}}, "c2" => %{eval: %{validity: :evaluated, status: :ready}}, - "c3" => %{eval: %{validity: :evaluated, status: :evaluating}} + "c3" => %{eval: %{validity: :fresh, status: :evaluating}} }, section_infos: %{ "s1" => %{evaluating_cell_id: nil, evaluation_queue: []}, @@ -2443,7 +2504,7 @@ defmodule Livebook.Session.DataTest do "c1" => %{eval: %{validity: :evaluated, status: :ready}}, "c2" => %{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: %{ "s1" => %{evaluating_cell_id: nil, evaluation_queue: []}, @@ -2544,7 +2605,7 @@ defmodule Livebook.Session.DataTest do "c1" => %{eval: %{validity: :evaluated, status: :ready}}, "c2" => %{eval: %{validity: :aborted, status: :ready}}, "c3" => %{eval: %{validity: :fresh, status: :ready}}, - "c4" => %{eval: %{validity: :evaluated, status: :evaluating}} + "c4" => %{eval: %{validity: :fresh, status: :evaluating}} }, section_infos: %{ "s1" => %{evaluating_cell_id: nil, evaluation_queue: []}, @@ -2622,7 +2683,7 @@ defmodule Livebook.Session.DataTest do 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) end @@ -3381,18 +3442,6 @@ defmodule Livebook.Session.DataTest do 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 test "returns an empty list when an invalid input id is given" do data = Data.new() diff --git a/test/livebook/session_test.exs b/test/livebook/session_test.exs index 2a0c8475d..d8b75eb37 100644 --- a/test/livebook/session_test.exs +++ b/test/livebook/session_test.exs @@ -1,8 +1,11 @@ defmodule Livebook.SessionTest do use ExUnit.Case, async: true + import Livebook.TestHelpers + alias Livebook.{Session, Delta, Runtime, Utils, Notebook, FileSystem} alias Livebook.Notebook.{Section, Cell} + alias Livebook.Session.Data setup do session = start_session() @@ -587,7 +590,7 @@ defmodule Livebook.SessionTest do 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 cell1 = %{Cell.new(:code) | id: "c1"} cell2 = %{Cell.new(:markdown) | id: "c2"} @@ -597,8 +600,9 @@ defmodule Livebook.SessionTest do section2 = %{Section.new() | id: "s2", cells: [cell3]} 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 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]} + 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 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]} 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 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]} + 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 diff --git a/test/support/noop_runtime.ex b/test/support/noop_runtime.ex index ef058c4cc..578d029fd 100644 --- a/test/support/noop_runtime.ex +++ b/test/support/noop_runtime.ex @@ -25,8 +25,8 @@ defmodule Livebook.Runtime.NoopRuntime do 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 end end diff --git a/test/support/test_helpers.ex b/test/support/test_helpers.ex index 20014a775..ea548d8e6 100644 --- a/test/support/test_helpers.ex +++ b/test/support/test_helpers.ex @@ -1,6 +1,8 @@ defmodule Livebook.TestHelpers do @moduledoc false + alias Livebook.Session.Data + @doc """ Creates file structure according to the given specification. """ @@ -18,4 +20,21 @@ defmodule Livebook.TestHelpers do 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