├── .tool-versions ├── guides ├── current-initiatives.md ├── quick-reads │ ├── users-perspective.md │ ├── recovery │ │ ├── proxy-startup.md │ │ ├── storage-recruitment.md │ │ ├── version-determination.md │ │ ├── resolver-startup.md │ │ ├── log-replay.md │ │ └── transaction-system-layout.md │ └── transaction-system-layout.md ├── deep-dives │ └── architecture │ │ ├── implementations │ │ └── README.md │ │ └── control-plane │ │ ├── director.md │ │ └── coordinator.md └── README.md ├── lib ├── bedrock │ ├── control_plane │ │ ├── rate_keeper.ex │ │ ├── config │ │ │ ├── log_descriptor.ex │ │ │ ├── service_descriptor.ex │ │ │ ├── resolver_descriptor.ex │ │ │ ├── policies.ex │ │ │ ├── storage_team_descriptor.ex │ │ │ └── parameters.ex │ │ ├── director │ │ │ ├── recovery │ │ │ │ ├── shared.ex │ │ │ │ ├── monitoring_phase.ex │ │ │ │ ├── recovery_phase.ex │ │ │ │ └── tsl_validation_phase.ex │ │ │ └── state.ex │ │ └── coordinator │ │ │ └── raft_adapter.ex │ ├── encoding.ex │ ├── encoding │ │ ├── binary.ex │ │ └── bert.ex │ ├── data_plane │ │ ├── log │ │ │ ├── shale.ex │ │ │ ├── shale │ │ │ │ ├── locking.ex │ │ │ │ ├── cold_starting.ex │ │ │ │ ├── facts.ex │ │ │ │ ├── state.ex │ │ │ │ └── long_pulls.ex │ │ │ └── telemetry.ex │ │ ├── storage │ │ │ ├── basalt.ex │ │ │ ├── olivine.ex │ │ │ ├── basalt │ │ │ │ ├── state.ex │ │ │ │ └── keyspace.ex │ │ │ └── olivine │ │ │ │ ├── id_allocator.ex │ │ │ │ └── state.ex │ │ ├── resolver │ │ │ ├── validation.ex │ │ │ └── state.ex │ │ ├── sequencer │ │ │ ├── telemetry.ex │ │ │ ├── state.ex │ │ │ └── tracing.ex │ │ ├── commit_proxy │ │ │ ├── state.ex │ │ │ ├── layout_optimization.ex │ │ │ ├── batch.ex │ │ │ └── telemetry.ex │ │ ├── proxy.ex │ │ ├── sequencer.ex │ │ ├── commit_proxy.ex │ │ └── resolver.ex │ ├── telemetry.ex │ ├── internal │ │ ├── gen_server_api.ex │ │ ├── time.ex │ │ ├── gen_server │ │ │ ├── replies.ex │ │ │ └── calls.ex │ │ └── timer_management.ex │ ├── service │ │ ├── foreman │ │ │ ├── health.ex │ │ │ ├── worker_info.ex │ │ │ ├── supervisor.ex │ │ │ ├── state.ex │ │ │ └── working_directory.ex │ │ ├── worker_behaviour.ex │ │ └── worker.ex │ ├── cluster │ │ ├── gateway │ │ │ ├── state.ex │ │ │ ├── transaction_builder │ │ │ │ ├── state.ex │ │ │ │ └── finalization.ex │ │ │ ├── worker_advertisement.ex │ │ │ ├── minimum_read_versions.ex │ │ │ ├── telemetry.ex │ │ │ └── tracing.ex │ │ └── gateway.ex │ ├── key_range.ex │ ├── type_coercion.ex │ ├── key.ex │ └── exceptions.ex ├── mix │ └── tasks │ │ ├── bedrock.ex │ │ ├── bedrock.config.ex │ │ └── bedrock.status.ex └── bedrock.ex ├── .formatter.exs ├── coveralls.json ├── test ├── bedrock │ ├── repo_test.exs │ ├── data_plane │ │ ├── log │ │ │ ├── shale_test.exs │ │ │ └── shale │ │ │ │ ├── locking_test.exs │ │ │ │ ├── cold_starting_test.exs │ │ │ │ ├── writer_test.exs │ │ │ │ ├── facts_test.exs │ │ │ │ ├── pull_boundary_test.exs │ │ │ │ ├── simple_pull_test.exs │ │ │ │ └── transaction_streams_test.exs │ │ ├── storage │ │ │ ├── olivine │ │ │ │ ├── tracing_test.exs │ │ │ │ └── index_tree_test.exs │ │ │ └── basalt_test.exs │ │ ├── commit_proxy │ │ │ └── sequencer_notification_test.exs │ │ └── transaction_atomic_test.exs │ ├── cluster │ │ └── gateway │ │ │ ├── fetch_coordinator_test.exs │ │ │ ├── transaction_builder │ │ │ └── no_write_conflict_test.exs │ │ │ └── discovery_test.exs │ ├── control_plane │ │ ├── coordinator │ │ │ ├── state_test.exs │ │ │ ├── api_test.exs │ │ │ ├── path_config_test.exs │ │ │ ├── end_epoch_test.exs │ │ │ └── integration_test.exs │ │ └── director │ │ │ ├── component_monitoring_test.exs │ │ │ └── recovery │ │ │ └── persistence_phase_test.exs │ ├── directory_integration_test.exs │ ├── integration │ │ └── brand_new_system_initialization_test.exs │ ├── directory │ │ └── metadata_test.exs │ └── internal │ │ └── atomics_test.exs ├── support │ ├── storage │ │ └── olivine │ │ │ ├── index_update_test_helpers.ex │ │ │ ├── test_helpers.ex │ │ │ └── index_manager_test_helpers.ex │ ├── gateway │ │ └── server_test_support.ex │ └── genserver_test_helpers_demo_test.exs └── test_helper.exs ├── .gitignore ├── README.md ├── .credo.exs ├── LICENSE ├── mix.exs └── .github └── workflows └── elixir_ci.yaml /.tool-versions: -------------------------------------------------------------------------------- 1 | elixir 1.18.4-otp-27 2 | erlang 27.3.3 3 | direnv 2.35.0 4 | -------------------------------------------------------------------------------- /guides/current-initiatives.md: -------------------------------------------------------------------------------- 1 | # Current Initiatives 2 | 3 | ## Adding support for Range Queries 4 | -------------------------------------------------------------------------------- /lib/bedrock/control_plane/rate_keeper.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.ControlPlane.RateKeeper do 2 | @moduledoc false 3 | end 4 | -------------------------------------------------------------------------------- /.formatter.exs: -------------------------------------------------------------------------------- 1 | # Used by "mix format" 2 | [ 3 | line_length: 120, 4 | inputs: ["{mix,.formatter}.exs", "{config,lib,test,benchmarks}/**/*.{ex,exs}"], 5 | plugins: [Styler] 6 | ] 7 | -------------------------------------------------------------------------------- /lib/bedrock/encoding.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.Encoding do 2 | @moduledoc false 3 | @callback pack(value :: any()) :: binary() 4 | @callback unpack(packed :: binary()) :: any() 5 | end 6 | -------------------------------------------------------------------------------- /lib/bedrock/encoding/binary.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.Encoding.None do 2 | @moduledoc false 3 | 4 | @behaviour Bedrock.Encoding 5 | 6 | def pack(value) when is_binary(value), do: value 7 | def unpack(packed), do: packed 8 | end 9 | -------------------------------------------------------------------------------- /lib/bedrock/encoding/bert.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.Encoding.BERT do 2 | @moduledoc false 3 | 4 | @behaviour Bedrock.Encoding 5 | 6 | def pack(value), do: :erlang.term_to_binary(value) 7 | def unpack(packed), do: :erlang.binary_to_term(packed) 8 | end 9 | -------------------------------------------------------------------------------- /coveralls.json: -------------------------------------------------------------------------------- 1 | { 2 | "coverage_options": { 3 | "treat_no_relevant_lines_as_covered": true, 4 | "minimum_coverage": 58.2 5 | }, 6 | "terminal_options": { 7 | "file_column_width": 80 8 | }, 9 | "skip_files": [ 10 | "test/support", 11 | ".github" 12 | ] 13 | } -------------------------------------------------------------------------------- /lib/bedrock/data_plane/log/shale.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.Log.Shale do 2 | @moduledoc false 3 | 4 | use Bedrock.Service.WorkerBehaviour, kind: :log 5 | 6 | alias Bedrock.DataPlane.Log.Shale.Server 7 | 8 | @doc false 9 | defdelegate child_spec(opts), to: Server 10 | end 11 | -------------------------------------------------------------------------------- /lib/bedrock/control_plane/config/log_descriptor.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.ControlPlane.Config.LogDescriptor do 2 | @moduledoc """ 3 | A `LogDescriptor` is a data structure that describes a log service within the 4 | system. 5 | """ 6 | 7 | @type vacancy :: {:vacancy, tag :: Bedrock.range_tag()} 8 | @type t :: [Bedrock.range_tag()] 9 | end 10 | -------------------------------------------------------------------------------- /test/bedrock/repo_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.RepoTest do 2 | use ExUnit.Case, async: true 3 | 4 | alias Bedrock.Repo 5 | 6 | defmodule TestCluster do 7 | @moduledoc false 8 | use Bedrock.Cluster, otp_app: :test_app, name: "test_cluster" 9 | end 10 | 11 | defmodule TestRepo do 12 | use Repo, cluster: TestCluster 13 | end 14 | end 15 | -------------------------------------------------------------------------------- /guides/quick-reads/users-perspective.md: -------------------------------------------------------------------------------- 1 | # Defining a Cluster 2 | 3 | ```elixir 4 | defmodule ExampleApp.Cluster do 5 | use ExampleApp.Cluster, 6 | otp_app: :example_app, 7 | name: "example" 8 | end 9 | 10 | ``` 11 | 12 | ## Executing a Transaction 13 | 14 | ```elixir 15 | # Basic transaction 16 | Repo.transact(fn -> 17 | value = Repo.get(key) 18 | Repo.put(key, new_value) 19 | {:ok, :ok} 20 | end) 21 | ``` 22 | -------------------------------------------------------------------------------- /lib/bedrock/control_plane/config/service_descriptor.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.ControlPlane.Config.ServiceDescriptor do 2 | @moduledoc false 3 | 4 | @type kind :: :log | :storage 5 | @type otp_name :: atom() 6 | @type status :: {:up, pid()} | :unknown | :down 7 | 8 | @type t :: %{ 9 | kind: kind(), 10 | last_seen: {otp_name(), node()} | nil, 11 | status: status() 12 | } 13 | end 14 | -------------------------------------------------------------------------------- /lib/bedrock/telemetry.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.Telemetry do 2 | @moduledoc false 3 | @spec execute( 4 | event_name :: [:bedrock | :storage | :log | :coordinator | atom()], 5 | measurements :: %{atom() => number()}, 6 | metadata :: %{atom() => term()} 7 | ) :: :ok 8 | def execute(event_name, measurements, metadata) do 9 | :telemetry.execute(event_name, measurements, metadata) 10 | end 11 | end 12 | -------------------------------------------------------------------------------- /guides/deep-dives/architecture/implementations/README.md: -------------------------------------------------------------------------------- 1 | 2 | # Implementation Components 3 | 4 | These are concrete storage engines that implement the data plane interfaces: 5 | 6 | - **[Basalt](basalt.md)** - An example multi-version storage engine based on :dets 7 | - **[Shale](shale.md)** - An example disk-based write-ahead-log implementation with segment file management 8 | 9 | Implementation components can be swapped or configured based on performance requirements, hardware characteristics, and deployment constraints. 10 | -------------------------------------------------------------------------------- /lib/bedrock/data_plane/storage/basalt.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.Storage.Basalt do 2 | @moduledoc false 3 | 4 | use Bedrock.Service.WorkerBehaviour, kind: :storage 5 | 6 | @doc false 7 | @spec child_spec( 8 | opts :: [ 9 | otp_name: atom(), 10 | foreman: Bedrock.Service.Foreman.ref(), 11 | id: Bedrock.service_id(), 12 | path: Path.t() 13 | ] 14 | ) :: Supervisor.child_spec() 15 | defdelegate child_spec(opts), to: __MODULE__.Server 16 | end 17 | -------------------------------------------------------------------------------- /lib/bedrock/data_plane/storage/olivine.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.Storage.Olivine do 2 | @moduledoc false 3 | 4 | use Bedrock.Service.WorkerBehaviour, kind: :storage 5 | 6 | @doc false 7 | @spec child_spec( 8 | opts :: [ 9 | otp_name: atom(), 10 | foreman: Bedrock.Service.Foreman.ref(), 11 | id: Bedrock.service_id(), 12 | path: Path.t() 13 | ] 14 | ) :: Supervisor.child_spec() 15 | defdelegate child_spec(opts), to: __MODULE__.Server 16 | end 17 | -------------------------------------------------------------------------------- /lib/bedrock/control_plane/config/resolver_descriptor.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.ControlPlane.Config.ResolverDescriptor do 2 | @moduledoc false 3 | 4 | @type t :: %{ 5 | start_key: Bedrock.key(), 6 | resolver: pid() | {:vacancy, non_neg_integer()} 7 | } 8 | 9 | @spec resolver_descriptor( 10 | start_key :: Bedrock.key(), 11 | resolver :: pid() | {:vacancy, non_neg_integer()} 12 | ) :: t() 13 | def resolver_descriptor(start_key, resolver), do: %{start_key: start_key, resolver: resolver} 14 | end 15 | -------------------------------------------------------------------------------- /lib/bedrock/internal/gen_server_api.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.Internal.GenServerApi do 2 | @moduledoc false 3 | 4 | defmacro __using__(opts) do 5 | module = Keyword.get(opts, :for) 6 | 7 | quote do 8 | import Bedrock.Internal.GenServer.Calls 9 | 10 | unquote do 11 | if module do 12 | quote do 13 | @doc false 14 | defdelegate child_spec(opts), to: unquote(module) 15 | 16 | def start_link(opts) do 17 | GenServer.start_link(unquote(module), opts) 18 | end 19 | end 20 | end 21 | end 22 | end 23 | end 24 | end 25 | -------------------------------------------------------------------------------- /test/bedrock/data_plane/log/shale_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.Log.ShaleTest do 2 | use ExUnit.Case, async: true 3 | 4 | alias Bedrock.DataPlane.Log.Shale 5 | alias Bedrock.DataPlane.Log.Shale.Server 6 | 7 | describe "child_spec/1" do 8 | test "delegates to Server.child_spec/1" do 9 | opts = [ 10 | cluster: %{}, 11 | otp_name: :test_log, 12 | foreman: self(), 13 | id: "test_log_1", 14 | path: "/tmp/test_log" 15 | ] 16 | 17 | # Test that child_spec is delegated to Server and has expected structure 18 | assert %{id: {Server, "test_log_1"}} = Shale.child_spec(opts) 19 | end 20 | end 21 | end 22 | -------------------------------------------------------------------------------- /lib/bedrock/service/foreman/health.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.Service.Foreman.Health do 2 | @moduledoc false 3 | 4 | alias Bedrock.Service.Foreman 5 | alias Bedrock.Service.Foreman.WorkerInfo 6 | 7 | @spec compute_health_from_worker_info([WorkerInfo.t()]) :: Foreman.health() 8 | def compute_health_from_worker_info(worker_info) do 9 | worker_info 10 | |> Enum.map(& &1.health) 11 | |> Enum.reduce(:ok, fn 12 | {:ok, _}, :ok -> :ok 13 | {:ok, _}, _ -> :starting 14 | {:failed_to_start, _}, :ok -> :starting 15 | {:failed_to_start, _}, _ -> {:failed_to_start, :at_least_one_failed_to_start} 16 | _, _ -> :unknown 17 | end) 18 | end 19 | end 20 | -------------------------------------------------------------------------------- /lib/bedrock/control_plane/config/policies.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.ControlPlane.Config.Policies do 2 | @moduledoc """ 3 | A `Policies` is a data structure that describes the policies that are used 4 | to configure the cluster. 5 | """ 6 | 7 | @typedoc """ 8 | Struct representing the policies that are used to configure the cluster. 9 | 10 | ## Fields: 11 | - `allow_volunteer_nodes_to_join` - Should nodes that volunteer to join the cluster be allowed to do so? 12 | """ 13 | @type t :: %{ 14 | allow_volunteer_nodes_to_join: boolean() 15 | } 16 | 17 | @spec default_policies() :: t() 18 | def default_policies, do: %{allow_volunteer_nodes_to_join: true} 19 | end 20 | -------------------------------------------------------------------------------- /lib/bedrock/data_plane/log/shale/locking.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.Log.Shale.Locking do 2 | @moduledoc false 3 | 4 | alias Bedrock.ControlPlane.Director 5 | alias Bedrock.DataPlane.Log.Shale.State 6 | 7 | @spec lock_for_recovery( 8 | t :: State.t(), 9 | epoch :: Bedrock.epoch(), 10 | director :: Director.ref() 11 | ) :: 12 | {:ok, State.t()} | {:error, :newer_epoch_exists} 13 | def lock_for_recovery(t, epoch, _director) when not is_nil(t.epoch) and epoch < t.epoch, 14 | do: {:error, :newer_epoch_exists} 15 | 16 | def lock_for_recovery(t, epoch, director) do 17 | {:ok, %{t | mode: :locked, epoch: epoch, director: director}} 18 | end 19 | end 20 | -------------------------------------------------------------------------------- /lib/bedrock/service/worker_behaviour.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.Service.WorkerBehaviour do 2 | @moduledoc false 3 | 4 | @callback kind() :: :log | :storage 5 | @callback one_time_initialization(Path.t()) :: :ok | {:error, File.posix()} 6 | 7 | defmacro __using__(opts) do 8 | kind = opts[:kind] || raise "Must declare a :kind" 9 | 10 | if kind not in [:log, :storage] do 11 | raise "Invalid :kind: #{inspect(kind)}" 12 | end 13 | 14 | quote do 15 | @behaviour Bedrock.Service.WorkerBehaviour 16 | 17 | @impl true 18 | def kind, do: unquote(kind) 19 | 20 | @impl true 21 | def one_time_initialization(_path), do: :ok 22 | 23 | defoverridable one_time_initialization: 1 24 | end 25 | end 26 | end 27 | -------------------------------------------------------------------------------- /test/support/storage/olivine/index_update_test_helpers.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.Test.Storage.Olivine.IndexUpdateTestHelpers do 2 | @moduledoc """ 3 | Test helper functions for IndexUpdate operations that are only used in tests. 4 | """ 5 | 6 | alias Bedrock.DataPlane.Storage.Olivine.IndexUpdate 7 | 8 | @doc """ 9 | Returns a list of {page_id, page_binary} tuples for pages that were modified during the update. 10 | This function is useful for test assertions and debugging. 11 | """ 12 | @spec modified_pages(IndexUpdate.t()) :: [{non_neg_integer(), binary()}] 13 | def modified_pages(%IndexUpdate{index: index, modified_page_ids: modified_page_ids}), 14 | do: Enum.map(modified_page_ids, fn page_id -> {page_id, index.page_map[page_id]} end) 15 | end 16 | -------------------------------------------------------------------------------- /test/bedrock/data_plane/log/shale/locking_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.Log.Shale.LockingTest do 2 | use ExUnit.Case 3 | 4 | alias Bedrock.DataPlane.Log.Shale.Locking 5 | alias Bedrock.DataPlane.Log.Shale.State 6 | 7 | describe "lock_for_recovery/3" do 8 | test "returns error when a newer epoch exists" do 9 | state = %State{epoch: 2} 10 | epoch = 1 11 | director = self() 12 | 13 | assert {:error, :newer_epoch_exists} = Locking.lock_for_recovery(state, epoch, director) 14 | end 15 | 16 | test "locks for recovery with valid epoch" do 17 | state = %State{epoch: nil, mode: :unlocked} 18 | epoch = 1 19 | director = self() 20 | 21 | assert {:ok, %State{mode: :locked, epoch: ^epoch, director: ^director}} = 22 | Locking.lock_for_recovery(state, epoch, director) 23 | end 24 | end 25 | end 26 | -------------------------------------------------------------------------------- /lib/bedrock/service/foreman/worker_info.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.Service.Foreman.WorkerInfo do 2 | @moduledoc false 3 | 4 | alias Bedrock.Service.Manifest 5 | alias Bedrock.Service.Worker 6 | 7 | @type health :: 8 | {:ok, Worker.ref()} 9 | | {:failed_to_start, File.posix() | :timeout | :already_started} 10 | | :stopped 11 | @type t :: %__MODULE__{} 12 | @enforce_keys [:id, :path, :health] 13 | 14 | defstruct [ 15 | :id, 16 | :path, 17 | :health, 18 | :manifest, 19 | :otp_name 20 | ] 21 | 22 | @spec put_health(t(), health()) :: t() 23 | def put_health(t, health), do: %{t | health: health} 24 | 25 | @spec put_manifest(t(), Manifest.t()) :: t() 26 | def put_manifest(t, manifest), do: %{t | manifest: manifest} 27 | 28 | @spec put_otp_name(t(), atom()) :: t() 29 | def put_otp_name(t, otp_name), do: %{t | otp_name: otp_name} 30 | end 31 | -------------------------------------------------------------------------------- /lib/mix/tasks/bedrock.ex: -------------------------------------------------------------------------------- 1 | defmodule Mix.Tasks.Bedrock do 2 | @shortdoc "Prints Bedrock help information" 3 | 4 | @moduledoc """ 5 | Prints Bedrock tasks and their information. 6 | 7 | $ mix bedrock 8 | 9 | """ 10 | 11 | use Mix.Task 12 | 13 | alias Mix.Tasks.Help 14 | 15 | @spec run([String.t()]) :: :ok 16 | def run(argv) do 17 | {_opts, args} = OptionParser.parse!(argv, strict: []) 18 | 19 | case args do 20 | [] -> general() 21 | _ -> Mix.raise("Invalid arguments, expected: mix ecto") 22 | end 23 | end 24 | 25 | defp general do 26 | Application.ensure_all_started(:bedrock) 27 | Mix.shell().info("Bedrock v#{Application.spec(:bedrock, :vsn)}") 28 | Mix.shell().info("A distributed ACID compliant key-value storage system for Elixir.") 29 | Mix.shell().info("\nAvailable tasks:\n") 30 | Help.run(["--search", "bedrock."]) 31 | end 32 | end 33 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | # Elixir LS 2 | /.elixir_ls/ 3 | 4 | # The directory Mix will write compiled artifacts to. 5 | /_build/ 6 | 7 | # If you run "mix test --cover", coverage assets end up here. 8 | /cover/ 9 | 10 | # The directory Mix downloads your dependencies sources to. 11 | /deps/ 12 | 13 | # Where third-party dependencies like ExDoc output generated docs. 14 | /doc/ 15 | 16 | # Ignore .fetch files in case you like to edit your project deps locally. 17 | /.fetch 18 | 19 | # If the VM crashes, it generates a dump, let's ignore it too. 20 | erl_crash.dump 21 | 22 | # Also ignore archive artifacts (built via "mix archive.build"). 23 | *.ez 24 | 25 | # Ignore package tarball (built via "mix hex.build"). 26 | bedrock-*.tar 27 | 28 | # Temporary files, for example, from tests. 29 | /tmp/ 30 | 31 | # Dialyzer PLT files. 32 | /priv/plts 33 | 34 | # MacOS cruft 35 | .DS_Store 36 | 37 | # Editor cruft 38 | .vscode/ 39 | 40 | # AI tools 41 | # .claude/ 42 | # CLAUDE.md 43 | -------------------------------------------------------------------------------- /lib/bedrock/internal/time.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.Internal.Time do 2 | @moduledoc """ 3 | Utility functions for retrieving the current system time in various formats. 4 | """ 5 | 6 | @doc "Return the current system time (UTC)." 7 | @spec now() :: DateTime.t() 8 | def now, do: DateTime.utc_now() 9 | 10 | @doc "Return the current system time in milliseconds." 11 | @spec now_in_ms() :: Bedrock.timestamp_in_ms() 12 | def now_in_ms, do: :os.system_time(:millisecond) 13 | 14 | @doc "Return the current system monotic time in milliseconds." 15 | @spec monotonic_now_in_ms() :: Bedrock.timestamp_in_ms() 16 | def monotonic_now_in_ms, do: :erlang.monotonic_time(:millisecond) 17 | 18 | @doc "Return the elapsed time in milliseconds since the given (monotonic) start time." 19 | @spec elapsed_monotonic_in_ms(Bedrock.timestamp_in_ms()) :: Bedrock.timestamp_in_ms() 20 | def elapsed_monotonic_in_ms(start_time), do: monotonic_now_in_ms() - start_time 21 | end 22 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # Bedrock 2 | 3 | [![Elixir CI](https://github.com/jallum/bedrock/actions/workflows/elixir_ci.yaml/badge.svg)](https://github.com/jallum/bedrock/actions/workflows/elixir_ci.yaml) 4 | [![Coverage Status](https://coveralls.io/repos/github/jallum/bedrock/badge.png?branch=develop)](https://coveralls.io/github/jallum/bedrock?branch=develop) 5 | 6 | Bedrock is an embedded, distributed key-value store with guarantees beyond ACID. 7 | It features consistent reads, strict serialization, transactions across the 8 | key-space and a simple API. 9 | 10 | ## Installation 11 | 12 | If [available in Hex](https://hex.pm/docs/publish), the package can be installed 13 | by adding `bedrock` to your list of dependencies in `mix.exs`: 14 | 15 | ```elixir 16 | def deps do 17 | [ 18 | {:bedrock, "~> 0.3"} 19 | ] 20 | end 21 | ``` 22 | 23 | ## Example 24 | 25 | [![Run in Livebook](https://livebook.dev/badge/v1/blue.svg)](https://livebook.dev/run?url=https%3A%2F%2Fraw.githubusercontent.com%2Fjallum%2Fbedrock%2Frefs%2Fheads%2Fdevelop%2Flivebooks%2Fclass_scheduling.livemd) 26 | -------------------------------------------------------------------------------- /.credo.exs: -------------------------------------------------------------------------------- 1 | %{ 2 | configs: [ 3 | %{ 4 | name: "default", 5 | files: %{ 6 | included: [ 7 | "lib/", 8 | "src/", 9 | "test/", 10 | "apps/*/lib/", 11 | "apps/*/src/", 12 | "apps/*/test/", 13 | ], 14 | excluded: [~r"/_build/", ~r"/deps/"] 15 | }, 16 | plugins: [], 17 | requires: [], 18 | strict: false, 19 | parse_timeout: 5000, 20 | color: true, 21 | checks: %{ 22 | 23 | disabled: [ 24 | # Disable the Logger metadata warning globally 25 | {Credo.Check.Warning.MissedMetadataKeyInLoggerConfig, []}, 26 | 27 | # Disable checks incompatible with Elixir 1.18.3 28 | {Credo.Check.Refactor.MapInto, []}, 29 | {Credo.Check.Warning.LazyLogging, []}, 30 | 31 | # Disable some checks that may be too strict for this project 32 | {Credo.Check.Readability.MultiAlias, []}, 33 | {Credo.Check.Readability.Specs, []} 34 | ] 35 | } 36 | } 37 | ] 38 | } 39 | -------------------------------------------------------------------------------- /test/bedrock/cluster/gateway/fetch_coordinator_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.Cluster.Gateway.FetchCoordinatorTest do 2 | use ExUnit.Case, async: true 3 | 4 | alias Bedrock.Cluster.Gateway.Server 5 | alias Bedrock.Cluster.Gateway.State 6 | 7 | describe "get_known_coordinator/0" do 8 | test "returns error when coordinator unavailable" do 9 | state = %State{ 10 | node: Node.self(), 11 | cluster: DefaultTestCluster, 12 | known_coordinator: :unavailable 13 | } 14 | 15 | assert {:reply, {:error, :unavailable}, ^state} = 16 | Server.handle_call(:get_known_coordinator, self(), state) 17 | end 18 | 19 | test "returns coordinator when available" do 20 | coordinator_ref = :test_coordinator_ref 21 | 22 | state = %State{ 23 | node: Node.self(), 24 | cluster: DefaultTestCluster, 25 | known_coordinator: coordinator_ref 26 | } 27 | 28 | assert {:reply, {:ok, ^coordinator_ref}, ^state} = 29 | Server.handle_call(:get_known_coordinator, self(), state) 30 | end 31 | end 32 | end 33 | -------------------------------------------------------------------------------- /test/bedrock/data_plane/storage/olivine/tracing_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.Storage.Olivine.TracingTest do 2 | use ExUnit.Case, async: true 3 | 4 | alias Bedrock.DataPlane.Storage.Olivine.Tracing 5 | 6 | describe "olivine tracing" do 7 | test "can start and stop tracing" do 8 | assert :ok = Tracing.start() 9 | assert {:error, :already_exists} = Tracing.start() 10 | assert :ok = Tracing.stop() 11 | assert :ok = Tracing.start() 12 | assert :ok = Tracing.stop() 13 | end 14 | 15 | test "handles olivine-specific telemetry events" do 16 | Tracing.start() 17 | 18 | # These should not crash 19 | Tracing.handler( 20 | [:bedrock, :storage, :transactions_queued], 21 | %{transaction_count: 5, queue_size: 10}, 22 | %{otp_name: :test_olivine}, 23 | nil 24 | ) 25 | 26 | Tracing.handler( 27 | [:bedrock, :storage, :transaction_processing_complete], 28 | %{batch_size: 3, duration_μs: 1500}, 29 | %{otp_name: :test_olivine}, 30 | nil 31 | ) 32 | 33 | Tracing.stop() 34 | end 35 | end 36 | end 37 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | MIT License 2 | 3 | Copyright (c) 2024-5 Jason Allum 4 | 5 | Permission is hereby granted, free of charge, to any person obtaining a copy 6 | of this software and associated documentation files (the "Software"), to deal 7 | in the Software without restriction, including without limitation the rights 8 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 9 | copies of the Software, and to permit persons to whom the Software is 10 | furnished to do so, subject to the following conditions: 11 | 12 | The above copyright notice and this permission notice shall be included in all 13 | copies or substantial portions of the Software. 14 | 15 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 16 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 17 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 18 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 19 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 20 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE 21 | SOFTWARE. 22 | -------------------------------------------------------------------------------- /lib/bedrock/data_plane/resolver/validation.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.Resolver.Validation do 2 | @moduledoc """ 3 | Validation functions for resolver transactions. 4 | 5 | Provides validation logic for transaction summaries and ensures they conform 6 | to the expected format before processing in the resolver. 7 | """ 8 | 9 | alias Bedrock.DataPlane.Transaction 10 | 11 | @spec check_transactions([Transaction.encoded()]) :: :ok | {:error, String.t()} 12 | def check_transactions(transactions) do 13 | cond do 14 | not is_list(transactions) -> 15 | {:error, "invalid transactions: expected list of binary transactions, got #{inspect(transactions)}"} 16 | 17 | not Enum.all?(transactions, &valid_transaction_summary?/1) -> 18 | {:error, "invalid transaction format: all transactions must be binary"} 19 | 20 | true -> 21 | :ok 22 | end 23 | end 24 | 25 | # Validation functions 26 | 27 | # Validates that a transaction is a proper binary transaction. 28 | @spec valid_transaction_summary?(any()) :: boolean() 29 | defp valid_transaction_summary?(transaction) when is_binary(transaction), do: true 30 | defp valid_transaction_summary?(_), do: false 31 | end 32 | -------------------------------------------------------------------------------- /lib/bedrock/data_plane/sequencer/telemetry.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.Sequencer.Telemetry do 2 | @moduledoc """ 3 | Telemetry utilities for sequencer operations. 4 | """ 5 | 6 | alias Bedrock.DataPlane.Version 7 | 8 | @spec emit_next_read_version(Version.t()) :: :ok 9 | def emit_next_read_version(version) do 10 | :telemetry.execute( 11 | [:bedrock, :sequencer, :next_read_version], 12 | %{}, 13 | %{version: version} 14 | ) 15 | end 16 | 17 | @spec emit_next_commit_version(Version.t(), Version.t(), non_neg_integer()) :: :ok 18 | def emit_next_commit_version(last_commit_version, commit_version, elapsed_us) do 19 | :telemetry.execute( 20 | [:bedrock, :sequencer, :next_commit_version], 21 | %{elapsed_us: elapsed_us}, 22 | %{last_commit_version: last_commit_version, commit_version: commit_version} 23 | ) 24 | end 25 | 26 | @spec emit_successful_commit(Version.t(), Version.t()) :: :ok 27 | def emit_successful_commit(commit_version, known_committed_version) do 28 | :telemetry.execute( 29 | [:bedrock, :sequencer, :successful_commit], 30 | %{}, 31 | %{commit_version: commit_version, known_committed_version: known_committed_version} 32 | ) 33 | end 34 | end 35 | -------------------------------------------------------------------------------- /lib/bedrock/service/foreman/supervisor.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.Service.Foreman.Supervisor do 2 | @moduledoc false 3 | 4 | @doc false 5 | @type foreman_opts :: [ 6 | cluster: module(), 7 | capabilities: [Bedrock.Cluster.capability()], 8 | path: Path.t() 9 | ] 10 | 11 | @spec child_spec(foreman_opts()) :: Supervisor.child_spec() 12 | def child_spec(opts) do 13 | cluster = Keyword.get(opts, :cluster) || raise "Missing :cluster option" 14 | capabilities = Keyword.get(opts, :capabilities) || raise "Missing :capabilities option" 15 | path = Keyword.get(opts, :path) || raise "Missing :path option" 16 | 17 | children = [ 18 | {DynamicSupervisor, name: cluster.otp_name(:worker_supervisor)}, 19 | {Bedrock.Service.Foreman.Server, 20 | [ 21 | cluster: cluster, 22 | capabilities: capabilities, 23 | path: path, 24 | otp_name: cluster.otp_name(:foreman) 25 | ]} 26 | ] 27 | 28 | %{ 29 | id: __MODULE__, 30 | start: { 31 | Supervisor, 32 | :start_link, 33 | [ 34 | children, 35 | [strategy: :one_for_one] 36 | ] 37 | }, 38 | restart: :permanent 39 | } 40 | end 41 | end 42 | -------------------------------------------------------------------------------- /lib/bedrock/data_plane/commit_proxy/state.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.CommitProxy.State do 2 | @moduledoc false 3 | 4 | alias Bedrock.ControlPlane.Config.TransactionSystemLayout 5 | alias Bedrock.DataPlane.CommitProxy.Batch 6 | alias Bedrock.DataPlane.CommitProxy.LayoutOptimization 7 | 8 | @type mode :: :locked | :running 9 | 10 | @type t :: %__MODULE__{ 11 | cluster: module(), 12 | director: pid(), 13 | transaction_system_layout: TransactionSystemLayout.t() | nil, 14 | precomputed_layout: LayoutOptimization.precomputed_layout() | nil, 15 | epoch: Bedrock.epoch(), 16 | batch: Batch.t() | nil, 17 | max_latency_in_ms: non_neg_integer(), 18 | max_per_batch: non_neg_integer(), 19 | empty_transaction_timeout_ms: non_neg_integer(), 20 | mode: mode(), 21 | lock_token: binary() 22 | } 23 | defstruct cluster: nil, 24 | director: nil, 25 | transaction_system_layout: nil, 26 | precomputed_layout: nil, 27 | epoch: nil, 28 | batch: nil, 29 | max_latency_in_ms: nil, 30 | max_per_batch: nil, 31 | empty_transaction_timeout_ms: nil, 32 | mode: :locked, 33 | lock_token: nil 34 | end 35 | -------------------------------------------------------------------------------- /lib/bedrock/data_plane/log/shale/cold_starting.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.Log.Shale.ColdStarting do 2 | @moduledoc false 3 | 4 | alias Bedrock.DataPlane.Log.Shale.Segment 5 | alias Bedrock.DataPlane.Version 6 | 7 | @spec reload_segments_at_path(segment_dir :: String.t()) :: 8 | {:ok, [Segment.t()]} 9 | | {:error, :unable_to_list_segments} 10 | def reload_segments_at_path(segment_dir) do 11 | segment_dir 12 | |> File.ls() 13 | |> case do 14 | {:ok, files} -> 15 | files 16 | |> Enum.filter(&String.starts_with?(&1, Segment.file_prefix())) 17 | |> Enum.map(fn file_name -> 18 | path = Path.join(segment_dir, file_name) 19 | 20 | min_version = 21 | file_name 22 | |> String.replace_prefix(Segment.file_prefix(), "") 23 | |> String.replace_suffix(".log", "") 24 | |> String.to_integer(32) 25 | |> Version.from_integer() 26 | 27 | {min_version, path} 28 | end) 29 | |> Enum.sort_by(&elem(&1, 0), :desc) 30 | |> Enum.map(fn {min_version, path} -> 31 | %Segment{min_version: min_version, path: path} 32 | end) 33 | |> then(&{:ok, &1}) 34 | 35 | {:error, _posix} -> 36 | {:error, :unable_to_list_segments} 37 | end 38 | end 39 | end 40 | -------------------------------------------------------------------------------- /lib/bedrock/data_plane/resolver/state.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.Resolver.State do 2 | @moduledoc """ 3 | State structure for Resolver GenServer processes. 4 | 5 | Maintains the interval tree for conflict detection, version tracking, and 6 | waiting queue for out-of-order transactions. Includes lock token for 7 | authentication. 8 | """ 9 | 10 | alias Bedrock.DataPlane.Resolver.Conflicts 11 | 12 | @type mode :: :running 13 | 14 | @type t :: %__MODULE__{ 15 | conflicts: Conflicts.t(), 16 | oldest_version: Bedrock.version(), 17 | last_version: Bedrock.version(), 18 | waiting: Bedrock.Internal.WaitingList.t(), 19 | mode: mode(), 20 | lock_token: Bedrock.lock_token(), 21 | epoch: Bedrock.epoch(), 22 | director: pid(), 23 | sweep_interval_ms: pos_integer(), 24 | version_retention_ms: pos_integer(), 25 | last_sweep_time: integer() 26 | } 27 | defstruct conflicts: nil, 28 | oldest_version: nil, 29 | last_version: nil, 30 | waiting: %{}, 31 | mode: :running, 32 | lock_token: nil, 33 | epoch: nil, 34 | director: nil, 35 | sweep_interval_ms: nil, 36 | version_retention_ms: nil, 37 | last_sweep_time: nil 38 | end 39 | -------------------------------------------------------------------------------- /lib/bedrock/internal/gen_server/replies.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.Internal.GenServer.Replies do 2 | @moduledoc """ 3 | Utilities for GenServer reply tuple construction. 4 | """ 5 | @spec reply(state, result) :: {:reply, result, state} when state: term(), result: term() 6 | def reply(t, result), do: {:reply, result, t} 7 | 8 | @spec reply(state, result, [{:continue, continue}]) :: 9 | {:reply, result, state, {:continue, continue}} 10 | when state: term(), result: term(), continue: term() 11 | def reply(t, result, continue: action), do: {:reply, result, t, {:continue, action}} 12 | 13 | @spec noreply( 14 | state, 15 | opts :: [ 16 | continue: continue, 17 | timeout: timeout() 18 | ] 19 | ) :: 20 | {:noreply, state} 21 | | {:noreply, state, {:continue, continue} | timeout()} 22 | when state: term(), continue: term() 23 | def noreply(t, opts \\ []) 24 | def noreply(t, continue: continue), do: {:noreply, t, {:continue, continue}} 25 | def noreply(t, timeout: ms), do: {:noreply, t, ms} 26 | def noreply(t, []), do: {:noreply, t} 27 | def noreply(_, opts), do: raise("Invalid options: #{inspect(opts)}") 28 | 29 | @spec stop(state, reason) :: {:stop, reason, state} when state: term(), reason: term() 30 | def stop(t, reason), do: {:stop, reason, t} 31 | end 32 | -------------------------------------------------------------------------------- /lib/bedrock/cluster/gateway/state.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.Cluster.Gateway.State do 2 | @moduledoc false 3 | 4 | alias Bedrock.Cluster.Descriptor 5 | alias Bedrock.ControlPlane.Config.TransactionSystemLayout 6 | alias Bedrock.ControlPlane.Coordinator 7 | 8 | @type t :: %__MODULE__{ 9 | node: node(), 10 | cluster: module(), 11 | path_to_descriptor: Path.t(), 12 | descriptor: Descriptor.t(), 13 | known_coordinator: Coordinator.ref() | :unavailable, 14 | timers: %{atom() => reference()} | nil, 15 | mode: :passive | :active, 16 | capabilities: [Bedrock.Cluster.capability()], 17 | transaction_system_layout: TransactionSystemLayout.t() | nil, 18 | # 19 | deadline_by_version: %{Bedrock.version() => Bedrock.timestamp_in_ms()}, 20 | minimum_read_version: Bedrock.version() | nil, 21 | lease_renewal_interval_in_ms: pos_integer() 22 | } 23 | defstruct node: nil, 24 | cluster: nil, 25 | path_to_descriptor: nil, 26 | descriptor: nil, 27 | known_coordinator: :unavailable, 28 | timers: nil, 29 | mode: :active, 30 | capabilities: [], 31 | transaction_system_layout: nil, 32 | # 33 | deadline_by_version: %{}, 34 | minimum_read_version: nil, 35 | lease_renewal_interval_in_ms: 5_000 36 | end 37 | -------------------------------------------------------------------------------- /lib/bedrock/cluster/gateway/transaction_builder/state.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.Cluster.Gateway.TransactionBuilder.State do 2 | @moduledoc false 3 | 4 | alias Bedrock.Cluster.Gateway.TransactionBuilder.LayoutIndex 5 | alias Bedrock.Cluster.Gateway.TransactionBuilder.Tx 6 | 7 | @type t :: %__MODULE__{ 8 | state: :valid | :committed | :rolled_back | :expired, 9 | gateway: pid(), 10 | transaction_system_layout: Bedrock.ControlPlane.Config.TransactionSystemLayout.t(), 11 | layout_index: LayoutIndex.t(), 12 | # 13 | read_version: Bedrock.version() | nil, 14 | read_version_lease_expiration: integer() | nil, 15 | commit_version: Bedrock.version() | nil, 16 | # 17 | tx: Tx.t(), 18 | stack: [Tx.t()], 19 | fastest_storage_servers: %{Bedrock.key_range() => pid()}, 20 | fetch_timeout_in_ms: pos_integer(), 21 | lease_renewal_threshold: pos_integer() 22 | } 23 | defstruct state: nil, 24 | gateway: nil, 25 | transaction_system_layout: nil, 26 | layout_index: nil, 27 | # 28 | read_version: nil, 29 | read_version_lease_expiration: nil, 30 | commit_version: nil, 31 | # 32 | tx: Tx.new(), 33 | stack: [], 34 | fastest_storage_servers: %{}, 35 | fetch_timeout_in_ms: 50, 36 | lease_renewal_threshold: 100, 37 | # 38 | active_range_queries: %{} 39 | end 40 | -------------------------------------------------------------------------------- /test/bedrock/control_plane/coordinator/state_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.ControlPlane.Coordinator.StateTest do 2 | use ExUnit.Case, async: true 3 | 4 | alias Bedrock.ControlPlane.Coordinator.State 5 | alias Bedrock.ControlPlane.Coordinator.State.Changes 6 | 7 | describe "service directory state changes" do 8 | test "put_service_directory replaces entire service directory" do 9 | initial_state = %State{ 10 | service_directory: %{"old" => {:storage, {:old_worker, :old_node@host}}} 11 | } 12 | 13 | new_directory = %{"new" => {:log, {:new_worker, :new_node@host}}} 14 | 15 | result = Changes.put_service_directory(initial_state, new_directory) 16 | 17 | assert result.service_directory == new_directory 18 | end 19 | 20 | test "update_service_directory applies updater function" do 21 | initial_state = %State{ 22 | service_directory: %{"service_1" => {:storage, {:worker1, :node1@host}}} 23 | } 24 | 25 | updater = fn directory -> 26 | directory 27 | |> Map.put("service_2", {:log, {:worker2, :node2@host}}) 28 | |> Map.delete("service_1") 29 | end 30 | 31 | result = Changes.update_service_directory(initial_state, updater) 32 | 33 | expected_directory = %{"service_2" => {:log, {:worker2, :node2@host}}} 34 | assert result.service_directory == expected_directory 35 | end 36 | 37 | test "state initializes with empty service directory" do 38 | state = %State{} 39 | assert state.service_directory == %{} 40 | end 41 | end 42 | end 43 | -------------------------------------------------------------------------------- /lib/bedrock/control_plane/director/recovery/shared.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.ControlPlane.Director.Recovery.Shared do 2 | @moduledoc """ 3 | Shared utilities for recovery phase modules. 4 | """ 5 | 6 | @type starter_fn :: (Supervisor.child_spec(), node() -> 7 | {:ok, pid()} 8 | | {:error, {:supervisor_exit, term()}} 9 | | {:error, {:supervisor_error, term()}} 10 | | {:error, {:unexpected_failure, term()}} 11 | | {:error, :already_started} 12 | | {:error, :max_children} 13 | | {:error, term()}) 14 | 15 | @doc """ 16 | Creates a starter function for supervised processes. 17 | 18 | Returns a function that can start child processes on specific nodes 19 | using the given supervisor OTP name. 20 | """ 21 | @spec starter_for(atom()) :: starter_fn() 22 | def starter_for(supervisor_otp_name) do 23 | fn child_spec, node -> 24 | try do 25 | {supervisor_otp_name, node} 26 | |> DynamicSupervisor.start_child(child_spec) 27 | |> case do 28 | {:ok, pid} -> {:ok, pid} 29 | {:ok, pid, _} -> {:ok, pid} 30 | {:error, {:already_started, pid}} -> {:ok, pid} 31 | {:error, reason} -> {:error, reason} 32 | end 33 | catch 34 | :exit, reason -> {:error, {:supervisor_exit, reason}} 35 | :error, reason -> {:error, {:supervisor_error, reason}} 36 | reason -> {:error, {:unexpected_failure, reason}} 37 | end 38 | end 39 | end 40 | end 41 | -------------------------------------------------------------------------------- /lib/bedrock/data_plane/storage/basalt/state.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.Storage.Basalt.State do 2 | @moduledoc false 3 | 4 | alias Bedrock.ControlPlane.Director 5 | alias Bedrock.DataPlane.Storage.Basalt.Database 6 | alias Bedrock.Internal.WaitingList 7 | alias Bedrock.Service.Foreman 8 | alias Bedrock.Service.Worker 9 | 10 | @type t :: %__MODULE__{ 11 | otp_name: atom(), 12 | path: Path.t(), 13 | foreman: Foreman.ref(), 14 | id: Worker.id(), 15 | database: Database.t(), 16 | pull_task: Task.t() | nil, 17 | epoch: Bedrock.epoch() | nil, 18 | director: Director.ref() | nil, 19 | mode: :locked | :running, 20 | waiting_fetches: WaitingList.t() 21 | } 22 | defstruct otp_name: nil, 23 | path: nil, 24 | foreman: nil, 25 | id: nil, 26 | database: nil, 27 | pull_task: nil, 28 | epoch: nil, 29 | director: nil, 30 | mode: :locked, 31 | waiting_fetches: %{} 32 | 33 | @spec update_mode(t(), :locked | :running) :: t() 34 | def update_mode(t, mode), do: %{t | mode: mode} 35 | 36 | @spec update_director_and_epoch(t(), Director.ref() | nil, Bedrock.epoch() | nil) :: t() 37 | def update_director_and_epoch(t, director, epoch), do: %{t | director: director, epoch: epoch} 38 | 39 | @spec reset_puller(t()) :: t() 40 | def reset_puller(t), do: %{t | pull_task: nil} 41 | 42 | @spec put_puller(t(), Task.t()) :: t() 43 | def put_puller(t, pull_task), do: %{t | pull_task: pull_task} 44 | end 45 | -------------------------------------------------------------------------------- /lib/mix/tasks/bedrock.config.ex: -------------------------------------------------------------------------------- 1 | defmodule Mix.Tasks.Bedrock.Config do 2 | @shortdoc "Prints configuration" 3 | 4 | @moduledoc """ 5 | Print configuration information about the given cluster. 6 | 7 | The clusters to interrogate are the ones specified under the 8 | `:bedrock_clusters` option in the current app configuration. However, 9 | if the `-c` option is given, it replaces the `:bedrock_clusters` config. 10 | 11 | Since Bedrock tasks can only be executed once, if you need to get status 12 | for multiple clusters, set `:bedrock_clusters` accordingly or pass the `-c` 13 | flag multiple times. 14 | 15 | ## Examples 16 | 17 | $ mix bedrock.config 18 | $ mix bedrock.config -r Custom.Cluster 19 | 20 | ## Command line options 21 | 22 | * `-c`, `--cluster` - the cluster to examine 23 | 24 | """ 25 | 26 | use Mix.Task 27 | 28 | import Mix.Bedrock 29 | 30 | defp switches, do: [cluster: [:string, :keep]] 31 | 32 | defp aliases, do: [c: :cluster] 33 | 34 | @spec run([String.t()]) :: :ok 35 | def run(argv) do 36 | {opts, _args} = OptionParser.parse!(argv, switches: switches(), aliases: aliases()) 37 | 38 | opts 39 | |> Keyword.get_values(:cluster) 40 | |> parse_clusters() 41 | |> Enum.each(fn cluster -> 42 | IO.puts( 43 | "\n#{pretty_name(cluster)} (#{cluster.name()}): #{inspect(cluster.config(), limit: :infinity, pretty: true)}" 44 | ) 45 | end) 46 | end 47 | 48 | @spec pretty_name(module()) :: String.t() 49 | def pretty_name(cluster) do 50 | cluster 51 | |> Module.split() 52 | |> Enum.join(".") 53 | end 54 | end 55 | -------------------------------------------------------------------------------- /lib/bedrock/data_plane/sequencer/state.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.Sequencer.State do 2 | @moduledoc """ 3 | State structure for Sequencer GenServer processes. 4 | 5 | Maintains Lamport clock state with microsecond-based version counters: 6 | - `known_committed_version_int`: Highest version confirmed as durably committed (for read requests) 7 | - `last_commit_version_int`: Last version handed to a commit proxy 8 | - `next_commit_version_int`: Next version to assign to a commit proxy 9 | - `epoch_baseline_version_int`: Starting point from previous epoch's last committed version 10 | - `epoch_start_monotonic_us`: Monotonic time when this epoch started 11 | 12 | Versions are generated using microsecond offsets from the epoch baseline, ensuring 13 | time-correlated version numbers while maintaining strict monotonic progression. 14 | The Lamport clock chain is formed by the pair (last_commit_version, next_commit_version). 15 | """ 16 | 17 | @type t() :: %__MODULE__{ 18 | director: pid(), 19 | epoch: Bedrock.epoch(), 20 | next_commit_version_int: integer(), 21 | last_commit_version_int: integer(), 22 | known_committed_version_int: integer(), 23 | epoch_baseline_version_int: integer(), 24 | epoch_start_monotonic_us: integer() 25 | } 26 | defstruct director: nil, 27 | epoch: 0, 28 | next_commit_version_int: nil, 29 | last_commit_version_int: nil, 30 | known_committed_version_int: nil, 31 | epoch_baseline_version_int: nil, 32 | epoch_start_monotonic_us: nil 33 | end 34 | -------------------------------------------------------------------------------- /test/bedrock/data_plane/log/shale/cold_starting_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.Log.Shale.ColdStartingTest do 2 | use ExUnit.Case, async: true 3 | 4 | alias Bedrock.DataPlane.Log.Shale.ColdStarting 5 | alias Bedrock.DataPlane.Log.Shale.Segment 6 | alias Bedrock.DataPlane.Version 7 | 8 | @segment_dir "test/fixtures/segments" 9 | 10 | setup do 11 | File.mkdir_p!(@segment_dir) 12 | on_exit(fn -> File.rm_rf!(@segment_dir) end) 13 | :ok 14 | end 15 | 16 | describe "reload_segments_at_path/2" do 17 | test "returns segments sorted by version in descending order" do 18 | create_segment_file(1) 19 | create_segment_file(2) 20 | create_segment_file(10) 21 | 22 | version_10 = Version.from_integer(10) 23 | version_2 = Version.from_integer(2) 24 | version_1 = Version.from_integer(1) 25 | 26 | assert {:ok, 27 | [ 28 | %Segment{min_version: ^version_10}, 29 | %Segment{min_version: ^version_2}, 30 | %Segment{min_version: ^version_1} 31 | ]} = ColdStarting.reload_segments_at_path(@segment_dir) 32 | end 33 | 34 | test "ignores non-matching files in directory" do 35 | create_segment_file(1) 36 | File.write!(Path.join(@segment_dir, "other_file.log"), "") 37 | 38 | version_1 = Version.from_integer(1) 39 | 40 | assert {:ok, [%Segment{min_version: ^version_1}]} = ColdStarting.reload_segments_at_path(@segment_dir) 41 | end 42 | end 43 | 44 | defp create_segment_file(version) do 45 | File.write!(Path.join(@segment_dir, Segment.encode_file_name(version)), "") 46 | end 47 | end 48 | -------------------------------------------------------------------------------- /lib/bedrock/data_plane/commit_proxy/layout_optimization.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.CommitProxy.LayoutOptimization do 2 | @moduledoc """ 3 | Precomputes expensive static structures from TransactionSystemLayout to optimize 4 | per-batch transaction processing performance. 5 | """ 6 | 7 | alias Bedrock.ControlPlane.Config.TransactionSystemLayout 8 | 9 | @max_key "\xff\xff\xff" 10 | 11 | @type precomputed_layout :: %{ 12 | resolver_ends: [{Bedrock.key(), pid()}], 13 | resolver_refs: [pid()] 14 | } 15 | @spec precompute_from_layout(TransactionSystemLayout.t()) :: precomputed_layout() 16 | def precompute_from_layout(%{resolvers: resolvers} = _layout) do 17 | resolver_list = resolvers 18 | resolver_ends = calculate_resolver_ends_optimized(resolver_list) 19 | resolver_refs = Enum.map(resolver_list, &elem(&1, 1)) 20 | 21 | %{ 22 | resolver_ends: resolver_ends, 23 | resolver_refs: resolver_refs 24 | } 25 | end 26 | 27 | @spec calculate_resolver_ends_optimized([{Bedrock.key(), any()}]) :: [{Bedrock.key(), pid()}] 28 | defp calculate_resolver_ends_optimized(resolvers) do 29 | sorted_resolvers = Enum.sort_by(resolvers, fn {start_key, _ref} -> start_key end) 30 | 31 | sorted_resolvers 32 | |> Enum.with_index() 33 | |> Enum.map(fn {{_start_key, resolver_ref}, index} -> 34 | max_key_ex = 35 | if index + 1 < length(sorted_resolvers) do 36 | {next_start, _} = Enum.at(sorted_resolvers, index + 1) 37 | next_start 38 | else 39 | @max_key 40 | end 41 | 42 | {max_key_ex, resolver_ref} 43 | end) 44 | end 45 | end 46 | -------------------------------------------------------------------------------- /test/bedrock/control_plane/coordinator/api_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.ControlPlane.Coordinator.ApiTest do 2 | use ExUnit.Case, async: true 3 | 4 | describe "service_info type validation" do 5 | test "validates service_info type" do 6 | # Test that the type is correctly defined 7 | service_info = {"service_id", :storage, {:worker, :node@host}} 8 | 9 | assert match?( 10 | {id, kind, {name, node}} 11 | when is_binary(id) and is_atom(kind) and is_atom(name) and is_atom(node), 12 | service_info 13 | ) 14 | end 15 | end 16 | 17 | describe "service registration API validation" do 18 | test "register_services validates service_info tuple format" do 19 | valid_services = [ 20 | {"service_1", :storage, {:worker1, :node1@host}}, 21 | {"service_2", :log, {:worker2, :node2@host}}, 22 | {"service_3", :sequencer, {:worker3, :node3@host}} 23 | ] 24 | 25 | # Verify each service matches the expected service_info pattern 26 | Enum.each(valid_services, fn service -> 27 | assert match?( 28 | {id, kind, {name, node}} 29 | when is_binary(id) and is_atom(kind) and is_atom(name) and is_atom(node), 30 | service 31 | ) 32 | end) 33 | end 34 | 35 | test "deregister_services validates service_id format" do 36 | valid_service_ids = ["service_1", "service_2", "service_3"] 37 | 38 | # Verify each service_id is a binary string 39 | Enum.each(valid_service_ids, fn service_id -> 40 | assert is_binary(service_id) 41 | end) 42 | end 43 | end 44 | end 45 | -------------------------------------------------------------------------------- /lib/bedrock/cluster/gateway/worker_advertisement.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.Cluster.Gateway.WorkerAdvertisement do 2 | @moduledoc """ 3 | Handles worker advertisement and service registration. 4 | Manages individual worker registration with coordinator via Raft consensus. 5 | """ 6 | 7 | alias Bedrock.Cluster.Gateway.State 8 | alias Bedrock.ControlPlane.Coordinator 9 | alias Bedrock.Service.Worker 10 | 11 | @spec advertise_worker_with_leader_check(State.t(), Worker.ref()) :: State.t() 12 | def advertise_worker_with_leader_check(%{known_coordinator: :unavailable} = t, _worker_pid) do 13 | # Ignore service notifications when no coordinator is available 14 | t 15 | end 16 | 17 | def advertise_worker_with_leader_check(t, worker_pid) do 18 | # Coordinator is available, process worker advertisement via coordinator 19 | register_single_worker_via_coordinator(worker_pid, t.known_coordinator) 20 | t 21 | end 22 | 23 | # Private helper functions 24 | 25 | @spec register_single_worker_via_coordinator( 26 | Worker.ref(), 27 | Coordinator.ref() | :unavailable 28 | ) :: :ok 29 | defp register_single_worker_via_coordinator(_worker_pid, :unavailable), do: :ok 30 | 31 | defp register_single_worker_via_coordinator(worker_pid, coordinator_ref) do 32 | case Worker.info(worker_pid, [:id, :otp_name, :kind, :pid]) do 33 | {:ok, worker_info} -> 34 | service_info = 35 | {worker_info[:id], worker_info[:kind], {worker_info[:otp_name], Node.self()}} 36 | 37 | Coordinator.register_services(coordinator_ref, [service_info]) 38 | :ok 39 | 40 | {:error, _} -> 41 | :ok 42 | end 43 | end 44 | end 45 | -------------------------------------------------------------------------------- /lib/bedrock/control_plane/config/storage_team_descriptor.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.ControlPlane.Config.StorageTeamDescriptor do 2 | @moduledoc false 3 | 4 | alias Bedrock.DataPlane.Storage 5 | 6 | @type vacancy :: {:vacancy, tag :: Bedrock.range_tag()} 7 | 8 | @typedoc """ 9 | ## Fields: 10 | - `tag`: The tag that identifies the team. 11 | - `key_range`: The range of keys that the team is responsible for. 12 | - `ids`: The list of storage workers that are responsible for the team. 13 | """ 14 | @type t :: %{ 15 | tag: Bedrock.range_tag(), 16 | key_range: Bedrock.key_range(), 17 | storage_ids: [Storage.id() | vacancy()] 18 | } 19 | 20 | @doc """ 21 | Create a new storage team descriptor. 22 | """ 23 | @spec storage_team_descriptor( 24 | Bedrock.range_tag(), 25 | Bedrock.key_range(), 26 | [Storage.id() | vacancy()] 27 | ) :: t() 28 | def storage_team_descriptor(tag, key_range, storage_ids), 29 | do: %{tag: tag, key_range: key_range, storage_ids: storage_ids} 30 | 31 | @doc """ 32 | Inserts a storage team descriptor into a list of storage team descriptors, 33 | replacing any existing storage team descriptor with the same tag. 34 | """ 35 | @spec upsert([t()], t()) :: [t()] 36 | def upsert([], n), do: [n] 37 | def upsert([%{tag: tag} | t], %{tag: tag} = n), do: [n | t] 38 | def upsert([h | t], n), do: [h | upsert(t, n)] 39 | 40 | @spec find_by_tag([t()], Bedrock.range_tag()) :: t() | nil 41 | def find_by_tag(l, tag), do: Enum.find(l, &(&1.tag == tag)) 42 | 43 | @spec remove_by_tag([t()], Bedrock.range_tag()) :: [t()] 44 | def remove_by_tag(l, tag), do: Enum.reject(l, &(&1.tag == tag)) 45 | end 46 | -------------------------------------------------------------------------------- /lib/mix/tasks/bedrock.status.ex: -------------------------------------------------------------------------------- 1 | defmodule Mix.Tasks.Bedrock.Status do 2 | @shortdoc "Prints status information" 3 | 4 | @moduledoc """ 5 | Print status information about the given cluster. 6 | 7 | The clusters to interrogate are the ones specified under the 8 | `:bedrock_clusters` option in the current app configuration. However, 9 | if the `-c` option is given, it replaces the `:bedrock_clusters` config. 10 | 11 | Since Bedrock tasks can only be executed once, if you need to get status 12 | for multiple clusters, set `:bedrock_clusters` accordingly or pass the `-c` 13 | flag multiple times. 14 | 15 | ## Examples 16 | 17 | $ mix bedrock.status 18 | $ mix bedrock.status -r Custom.Cluster 19 | 20 | ## Command line options 21 | 22 | * `-c`, `--cluster` - the cluster to examine 23 | 24 | """ 25 | 26 | use Mix.Task 27 | 28 | import Mix.Bedrock 29 | 30 | defp switches, do: [cluster: [:string, :keep]] 31 | 32 | defp aliases, do: [c: :cluster] 33 | 34 | @spec run([String.t()]) :: :ok 35 | def run(argv) do 36 | {opts, _args} = OptionParser.parse!(argv, switches: switches(), aliases: aliases()) 37 | 38 | Application.ensure_all_started(:bedrock) 39 | 40 | clusters = 41 | opts 42 | |> Keyword.get_values(:cluster) 43 | |> parse_clusters() 44 | 45 | {:ok, _pid} = 46 | clusters 47 | |> Supervisor.start_link(strategy: :one_for_one) 48 | |> case do 49 | {:ok, pid} -> {:ok, pid} 50 | {:error, {:already_started, pid}} -> {:ok, pid} 51 | end 52 | 53 | for cluster <- clusters do 54 | case cluster.config() do 55 | {:ok, config} -> config 56 | {:error, reason} -> IO.puts("Error: #{reason}") 57 | end 58 | end 59 | 60 | :ok 61 | end 62 | end 63 | -------------------------------------------------------------------------------- /lib/bedrock/cluster/gateway/minimum_read_versions.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.Cluster.Gateway.MinimumReadVersions do 2 | @moduledoc false 3 | 4 | alias Bedrock.Cluster.Gateway.State 5 | alias Bedrock.Internal.Time 6 | 7 | @type deadline_by_version :: %{Bedrock.version() => Bedrock.timestamp_in_ms()} 8 | 9 | @spec recalculate_minimum_read_version(State.t()) :: State.t() 10 | def recalculate_minimum_read_version(t) do 11 | {minimum_read_version, deadline_by_version} = 12 | find_minimum_active_version( 13 | t.deadline_by_version, 14 | Time.monotonic_now_in_ms() 15 | ) 16 | 17 | t 18 | |> Map.put(:deadline_by_version, deadline_by_version) 19 | |> Map.put(:minimum_read_version, minimum_read_version) 20 | end 21 | 22 | @doc """ 23 | Find the minimum active version with a deadline greater than the current time. 24 | 25 | Iterates over the `deadline_by_version` map to determine the minimum version with 26 | a deadline in the future. Returns a tuple containing this minimum version and an 27 | updated map of versions which still have valid deadlines. It returns `nil` if 28 | no versions have a deadline in the future and an empty map if all deadlines 29 | have expired. 30 | """ 31 | @spec find_minimum_active_version(deadline_by_version(), at :: Bedrock.timestamp_in_ms()) :: 32 | {minimum_version :: Bedrock.version() | nil, deadline_by_version()} 33 | def find_minimum_active_version(deadline_by_version, at) do 34 | Enum.reduce(deadline_by_version, {nil, %{}}, fn 35 | {read_version, deadline}, {minimum_version, deadline_by_version} when deadline > at -> 36 | {min(read_version, minimum_version), Map.put(deadline_by_version, read_version, deadline)} 37 | 38 | _, acc -> 39 | acc 40 | end) 41 | end 42 | end 43 | -------------------------------------------------------------------------------- /lib/bedrock/data_plane/proxy.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.Proxy do 2 | @moduledoc """ 3 | Read version proxy for Bedrock's transaction system. 4 | """ 5 | use GenServer 6 | 7 | alias Bedrock.ControlPlane.Config.TransactionSystemLayout 8 | alias Bedrock.ControlPlane.Director 9 | alias Bedrock.DataPlane.Sequencer 10 | 11 | @type t :: %__MODULE__{ 12 | director: Director.ref(), 13 | layout: TransactionSystemLayout.t() 14 | } 15 | 16 | defstruct ~w[director layout]a 17 | 18 | @spec next_read_version(pid() | atom()) :: Bedrock.version() 19 | def next_read_version(proxy), do: GenServer.call(proxy, :get_read_version) 20 | 21 | @spec child_spec(opts :: [id: String.t(), director: Director.ref(), layout: TransactionSystemLayout.t()]) :: 22 | Supervisor.child_spec() 23 | def child_spec(opts) do 24 | id = Keyword.get(opts, :id) || raise "Missing :id option" 25 | director = Keyword.get(opts, :director) || raise "Missing :director option" 26 | layout = Keyword.get(opts, :layout) || raise "Missing :layout option" 27 | 28 | %{ 29 | id: id, 30 | start: {GenServer, :start_link, [__MODULE__, {director, layout}]}, 31 | restart: :transient 32 | } 33 | end 34 | 35 | @impl GenServer 36 | def init({director, layout}) do 37 | {:ok, 38 | %__MODULE__{ 39 | director: director, 40 | layout: layout 41 | }} 42 | end 43 | 44 | @impl GenServer 45 | def handle_call(:get_read_version, from, state) do 46 | forward_call(state.layout.sequencer, from, :next_read_version) 47 | {:noreply, state} 48 | end 49 | 50 | @spec forward_call(Sequencer.ref(), GenServer.from(), :next_read_version) :: 51 | :ok 52 | defp forward_call(gen_server, from, message), do: send(gen_server, {:"$gen_call", from, message}) 53 | end 54 | -------------------------------------------------------------------------------- /lib/bedrock/data_plane/storage/olivine/id_allocator.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.Storage.Olivine.IdAllocator do 2 | @moduledoc """ 3 | Manages ID allocation and reallocation for pages. 4 | """ 5 | 6 | alias Bedrock.DataPlane.Storage.Olivine.Index.Page 7 | 8 | @type t :: %__MODULE__{ 9 | max_id: Page.id(), 10 | free_ids: [Page.id()] 11 | } 12 | 13 | defstruct [:max_id, :free_ids] 14 | 15 | @spec new(Page.id(), [Page.id()]) :: t() 16 | def new(max_id, free_ids) do 17 | %__MODULE__{max_id: max_id, free_ids: free_ids} 18 | end 19 | 20 | @spec allocate_id(t()) :: {Page.id(), t()} 21 | def allocate_id(%__MODULE__{free_ids: [id | rest]} = allocator) do 22 | {id, %{allocator | free_ids: rest}} 23 | end 24 | 25 | def allocate_id(%__MODULE__{free_ids: [], max_id: max_id} = allocator) do 26 | new_id = max_id + 1 27 | {new_id, %{allocator | max_id: new_id}} 28 | end 29 | 30 | @spec allocate_ids(t(), non_neg_integer()) :: {[Page.id()], t()} 31 | def allocate_ids(allocator, count) when count <= 0, do: {[], allocator} 32 | 33 | def allocate_ids(allocator, count) do 34 | 1..count 35 | |> Enum.reduce({[], allocator}, fn _, {ids_acc, allocator_acc} -> 36 | {new_id, updated_allocator} = allocate_id(allocator_acc) 37 | {[new_id | ids_acc], updated_allocator} 38 | end) 39 | |> then(fn {ids, final_allocator} -> {Enum.reverse(ids), final_allocator} end) 40 | end 41 | 42 | @spec recycle_id(t(), Page.id()) :: t() 43 | def recycle_id(%__MODULE__{free_ids: free_ids} = allocator, page_id), 44 | do: %{allocator | free_ids: [page_id | free_ids]} 45 | 46 | @spec recycle_ids(t(), [Page.id()]) :: t() 47 | def recycle_ids(%__MODULE__{free_ids: free_ids} = allocator, page_ids), 48 | do: %{allocator | free_ids: page_ids ++ free_ids} 49 | end 50 | -------------------------------------------------------------------------------- /lib/bedrock/internal/timer_management.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.Internal.TimerManagement do 2 | @moduledoc """ 3 | Utilities for managing GenServer timers in a structured way. 4 | """ 5 | @type timer_container :: %{:timers => %{atom() => reference()} | nil, optional(atom()) => any()} 6 | defmacro __using__(_opts) do 7 | quote do 8 | import Bedrock.Internal.TimerManagement, 9 | only: [cancel_all_timers: 1, cancel_timer: 2, set_timer: 3] 10 | end 11 | end 12 | 13 | # Timer Management 14 | 15 | @spec cancel_all_timers(timer_container()) :: timer_container() 16 | def cancel_all_timers(%{timers: nil} = t), do: t 17 | 18 | def cancel_all_timers(%{} = t) do 19 | update_in(t.timers, fn timers -> 20 | Enum.each(timers, &(&1 |> elem(1) |> Process.cancel_timer())) 21 | %{} 22 | end) 23 | end 24 | 25 | @spec cancel_timer(timer_container(), timer_name :: atom()) :: timer_container() 26 | def cancel_timer(%{timers: nil} = t, _name), do: t 27 | 28 | def cancel_timer(%{} = t, name) do 29 | {timer_ref, timers} = Map.pop(t.timers, name) 30 | 31 | case timer_ref do 32 | nil -> 33 | t 34 | 35 | _ -> 36 | Process.cancel_timer(timer_ref) 37 | put_in(t.timers, timers) 38 | end 39 | end 40 | 41 | @spec set_timer(timer_container(), timer_name :: atom(), timeout_ms :: pos_integer()) :: 42 | timer_container() 43 | def set_timer(%{} = t, name, timeout_in_ms) do 44 | update_in( 45 | t.timers, 46 | fn 47 | %{^name => _} -> 48 | raise("Timer #{name} already set") 49 | 50 | timers -> 51 | Map.put( 52 | timers || %{}, 53 | name, 54 | Process.send_after(self(), {:timeout, name}, timeout_in_ms) 55 | ) 56 | end 57 | ) 58 | end 59 | end 60 | -------------------------------------------------------------------------------- /lib/bedrock/data_plane/sequencer/tracing.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.Sequencer.Tracing do 2 | @moduledoc false 3 | alias Bedrock.DataPlane.Version 4 | 5 | require Logger 6 | 7 | @spec handler_id() :: String.t() 8 | defp handler_id, do: "bedrock_trace_data_plane_sequencer" 9 | 10 | @spec start() :: :ok | {:error, :already_exists} 11 | def start do 12 | :telemetry.attach_many( 13 | handler_id(), 14 | [ 15 | [:bedrock, :sequencer, :next_read_version], 16 | [:bedrock, :sequencer, :next_commit_version], 17 | [:bedrock, :sequencer, :successful_commit] 18 | ], 19 | &__MODULE__.handler/4, 20 | nil 21 | ) 22 | end 23 | 24 | @spec stop() :: :ok | {:error, :not_found} 25 | def stop, do: :telemetry.detach(handler_id()) 26 | 27 | @spec handler(list(atom()), map(), map(), term()) :: :ok 28 | def handler([:bedrock, :sequencer, event], measurements, metadata, _), do: log_event(event, measurements, metadata) 29 | 30 | @spec log_event(atom(), map(), map()) :: :ok 31 | def log_event(:next_read_version, _measurements, metadata) do 32 | info("Next read version: #{Version.to_string(metadata.version)}") 33 | end 34 | 35 | def log_event(:next_commit_version, measurements, metadata) do 36 | info( 37 | "Next commit version: #{Version.to_string(metadata.last_commit_version)} -> #{Version.to_string(metadata.commit_version)} (elapsed: #{measurements.elapsed_us}μs)" 38 | ) 39 | end 40 | 41 | def log_event(:successful_commit, _measurements, metadata) do 42 | info( 43 | "Successful commit reported: #{Version.to_string(metadata.commit_version)}, known committed: #{Version.to_string(metadata.known_committed_version)}" 44 | ) 45 | end 46 | 47 | defp info(message) do 48 | Logger.info("Bedrock Sequencer: #{message}", ansi_color: :green) 49 | end 50 | end 51 | -------------------------------------------------------------------------------- /test/bedrock/directory_integration_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DirectoryIntegrationTest do 2 | use ExUnit.Case, async: true 3 | 4 | import Bedrock.Test.DirectoryHelpers 5 | import Mox 6 | 7 | alias Bedrock.Directory 8 | 9 | setup do 10 | stub(MockRepo, :transact, fn callback -> callback.() end) 11 | :ok 12 | end 13 | 14 | setup :verify_on_exit! 15 | 16 | # Helper for deterministic prefix allocator 17 | defp create_prefix_allocator do 18 | {:ok, agent} = Agent.start_link(fn -> 0 end) 19 | 20 | next_prefix_fn = fn -> 21 | Agent.get_and_update(agent, fn n -> 22 | {<>, n + 1} 23 | end) 24 | end 25 | 26 | {agent, next_prefix_fn} 27 | end 28 | 29 | test "full directory workflow" do 30 | {_agent, next_prefix_fn} = create_prefix_allocator() 31 | 32 | # Use proper expectations with directory helpers 33 | MockRepo 34 | |> expect_version_initialization() 35 | |> expect_directory_exists(["users"], nil) 36 | |> expect_directory_creation(["users"], {<<0, 0, 0, 0>>, "document"}) 37 | 38 | # Create directory layer 39 | root = Directory.root(MockRepo, next_prefix_fn: next_prefix_fn) 40 | 41 | # Execute the workflow with pattern matching assertions 42 | # Root is already created by Directory.root(), verify its properties 43 | assert %Directory.Node{path: [], prefix: ""} = root 44 | 45 | assert {:ok, users_dir} = Directory.create(root, ["users"], layer: "document") 46 | assert %{layer: "document", path: ["users"], prefix: <<0, 0, 0, 0>>} = users_dir 47 | 48 | # Test keyspace generation uses the correct base prefix 49 | keyspace = users_dir |> Directory.to_keyspace() |> Bedrock.Keyspace.partition("data") 50 | # Should start with the users_dir prefix 51 | assert String.starts_with?(keyspace.prefix, <<0, 0, 0, 0>>) 52 | end 53 | end 54 | -------------------------------------------------------------------------------- /lib/bedrock/internal/gen_server/calls.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.Internal.GenServer.Calls do 2 | @moduledoc """ 3 | Utilities for GenServer call and cast operations. 4 | """ 5 | @spec broadcast([node()], otp_name :: atom(), message :: term()) :: :ok 6 | def broadcast(nodes, otp_name, message) do 7 | GenServer.abcast(nodes, otp_name, message) 8 | :ok 9 | end 10 | 11 | @spec cast(GenServer.server(), message :: term()) :: :ok 12 | def cast(server, message), do: GenServer.cast(server, message) 13 | 14 | @spec call(GenServer.server(), message :: term(), timeout()) :: 15 | term() 16 | | {:error, :unavailable} 17 | | {:error, :timeout} 18 | | {:error, :unknown} 19 | def call(server, message, timeout) do 20 | GenServer.call(server, message, normalize_timeout(timeout)) 21 | rescue 22 | _ -> {:error, :unknown} 23 | catch 24 | :exit, {:noproc, _} -> {:error, :unavailable} 25 | :exit, {{:nodedown, _}, _} -> {:error, :unavailable} 26 | :exit, {:timeout, _} -> {:error, :timeout} 27 | end 28 | 29 | @spec call!(GenServer.server(), message :: term(), timeout()) :: term() 30 | def call!(server, message, timeout) do 31 | GenServer.call(server, message, normalize_timeout(timeout)) 32 | rescue 33 | error -> raise "GenServer call failed: #{inspect(error)}" 34 | catch 35 | :exit, {:noproc, _} -> raise "GenServer call failed: #{inspect(:unavailable)}" 36 | :exit, {{:nodedown, _}, _} -> raise "GenServer call failed: #{inspect(:unavailable)}" 37 | :exit, {:timeout, _} -> raise "GenServer call failed: #{inspect(:timeout)}" 38 | end 39 | 40 | @spec normalize_timeout(timeout()) :: timeout() 41 | defp normalize_timeout(:infinity), do: :infinity 42 | defp normalize_timeout(timeout) when is_integer(timeout) and timeout >= 0, do: timeout 43 | defp normalize_timeout(_), do: 5000 44 | end 45 | -------------------------------------------------------------------------------- /lib/bedrock/data_plane/log/shale/facts.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.Log.Shale.Facts do 2 | @moduledoc false 3 | alias Bedrock.DataPlane.Log 4 | alias Bedrock.DataPlane.Log.Shale.State 5 | 6 | @spec info(State.t(), Log.fact_name()) :: {:ok, term()} | {:error, :unsupported} 7 | @spec info(State.t(), [Log.fact_name()]) :: {:ok, %{Log.fact_name() => term()}} 8 | def info(%State{} = t, fact) when is_atom(fact) do 9 | case gather_info(fact, t) do 10 | {:error, _reason} = error -> error 11 | info -> {:ok, info} 12 | end 13 | end 14 | 15 | def info(%State{} = t, facts) when is_list(facts) do 16 | {:ok, 17 | Map.new(facts, fn 18 | fact_name -> {fact_name, gather_info(fact_name, t)} 19 | end)} 20 | end 21 | 22 | @spec supported_info() :: [Log.fact_name()] 23 | def supported_info, 24 | do: [:id, :kind, :minimum_durable_version, :oldest_version, :last_version, :otp_name, :pid, :state, :supported_info] 25 | 26 | @spec gather_info(Log.fact_name(), State.t()) :: 27 | String.t() 28 | | :log 29 | | atom() 30 | | pid() 31 | | [Log.fact_name()] 32 | | :unavailable 33 | | Bedrock.version() 34 | | {:error, :unsupported} 35 | # Worker facts 36 | defp gather_info(:id, %{id: id}), do: id 37 | defp gather_info(:kind, _t), do: :log 38 | defp gather_info(:otp_name, %State{otp_name: otp_name}), do: otp_name 39 | defp gather_info(:pid, _), do: self() 40 | defp gather_info(:supported_info, _), do: supported_info() 41 | 42 | # Transaction Log facts 43 | defp gather_info(:minimum_durable_version, _t), do: :unavailable 44 | 45 | defp gather_info(:oldest_version, t), do: t.oldest_version 46 | defp gather_info(:last_version, t), do: t.last_version 47 | 48 | # Everything else... 49 | defp gather_info(_, _), do: {:error, :unsupported} 50 | end 51 | -------------------------------------------------------------------------------- /test/bedrock/data_plane/log/shale/writer_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.Log.Shale.WriterTest do 2 | use ExUnit.Case, async: true 3 | 4 | alias Bedrock.DataPlane.Log.Shale.Writer 5 | 6 | @test_file "test_segment.log" 7 | 8 | setup do 9 | File.write!(@test_file, :binary.copy(<<0>>, 1024)) 10 | on_exit(fn -> File.rm(@test_file) end) 11 | {:ok, writer} = Writer.open(@test_file) 12 | %{writer: writer} 13 | end 14 | 15 | describe "open/1" do 16 | test "successfully opens a file and initializes the writer struct" do 17 | # Use pattern matching to assert all fields and that fd is present 18 | assert {:ok, %Writer{fd: fd, write_offset: 4, bytes_remaining: 1004}} = 19 | Writer.open(@test_file) 20 | 21 | assert fd 22 | end 23 | end 24 | 25 | describe "close/1" do 26 | test "successfully closes the file descriptor", %{writer: writer} do 27 | assert :ok = Writer.close(writer) 28 | end 29 | 30 | test "returns :ok when writer is nil" do 31 | assert :ok = Writer.close(nil) 32 | end 33 | end 34 | 35 | describe "append/2" do 36 | test "returns :segment_full error when there is not enough space", %{writer: writer} do 37 | large_transaction = :binary.copy(<<0>>, 1016) 38 | commit_version = <<1::unsigned-big-64>> 39 | assert {:error, :segment_full} = Writer.append(writer, large_transaction, commit_version) 40 | end 41 | 42 | test "successfully appends a transaction and updates the writer struct", %{writer: writer} do 43 | transaction = <<1, 2, 3, 4>> 44 | commit_version = <<1::unsigned-big-64>> 45 | 46 | # Use pattern matching to assert the exact structure and values 47 | assert {:ok, %Writer{write_offset: 24, bytes_remaining: 984}} = 48 | Writer.append(writer, transaction, commit_version) 49 | end 50 | end 51 | end 52 | -------------------------------------------------------------------------------- /lib/bedrock/service/worker.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.Service.Worker do 2 | @moduledoc """ 3 | A worker is a GenServer that is started and stopped by a service director. 4 | It is expected to provide a set of facts about itself when requested along 5 | with other services (as befits the type of worker.) 6 | """ 7 | 8 | use Bedrock.Internal.GenServerApi 9 | 10 | @type ref :: pid() | atom() | {atom(), node()} 11 | @type id :: Bedrock.service_id() 12 | @type fact_name :: :supported_info | :kind | :id | :health | :otp_name | :pid 13 | @type fact_value :: 14 | [fact_name()] | :log | :storage | Bedrock.service_id() | health() | otp_name() | pid() 15 | @type timeout_in_ms :: Bedrock.timeout_in_ms() 16 | @type health :: {:ok, pid()} | :stopped | {:error, :timeout | :unavailable} 17 | @type otp_name :: atom() 18 | 19 | @spec random_id() :: binary() 20 | def random_id, do: 5 |> :crypto.strong_rand_bytes() |> Base.encode32(case: :lower) 21 | 22 | @spec info( 23 | worker_ref :: ref(), 24 | requested_facts :: [fact_name() | atom()], 25 | opts :: [timeout_in_ms: timeout_in_ms()] 26 | ) :: 27 | {:ok, facts :: %{fact_name() => fact_value()}} | {:error, :unavailable} 28 | def info(worker, fact_names, opts \\ []), do: call(worker, {:info, fact_names}, opts[:timeout_in_ms] || :infinity) 29 | 30 | @spec lock_for_recovery( 31 | worker_ref :: ref(), 32 | recovery_epoch :: Bedrock.epoch(), 33 | opts :: [timeout_in_ms: timeout_in_ms()] 34 | ) :: 35 | {:ok, worker_pid :: pid(), recovery_info :: [kind: :log | :storage, version: Bedrock.version()]} 36 | | {:error, :newer_epoch_exists} 37 | | {:error, :timeout} 38 | def lock_for_recovery(worker, epoch, opts \\ []), 39 | do: call(worker, {:lock_for_recovery, epoch}, opts[:timeout_in_ms] || :infinity) 40 | end 41 | -------------------------------------------------------------------------------- /test/bedrock/data_plane/storage/olivine/index_tree_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.Storage.Olivine.Index.TreeTest do 2 | use ExUnit.Case, async: true 3 | 4 | alias Bedrock.DataPlane.Storage.Olivine.Index.Page 5 | alias Bedrock.DataPlane.Storage.Olivine.Index.Tree 6 | 7 | describe "page_for_key/2" do 8 | test "returns correct page for key insertion" do 9 | page1_kvs = [{"a", <<1::64>>}, {"f", <<2::64>>}] 10 | page2_kvs = [{"g", <<3::64>>}, {"m", <<4::64>>}] 11 | 12 | page1 = Page.new(1, page1_kvs) 13 | page2 = Page.new(2, page2_kvs) 14 | 15 | tree = 16 | :gb_trees.empty() 17 | |> Tree.add_page_to_tree(page1) 18 | |> Tree.add_page_to_tree(page2) 19 | 20 | # Key "c" should go to page 1 (contains "a" to "f") 21 | assert Tree.page_for_key(tree, "c") == 1 22 | 23 | # Key "j" should go to page 2 (contains "g" to "m") 24 | assert Tree.page_for_key(tree, "j") == 2 25 | 26 | # Key "z" (beyond all pages) should go to rightmost page (always 0) 27 | assert Tree.page_for_key(tree, "z") == 0 28 | end 29 | end 30 | 31 | describe "add_page_to_tree/2" do 32 | test "adds page correctly to tree" do 33 | page_kvs = [{"key1", <<1::64>>}, {"key3", <<2::64>>}] 34 | page = Page.new(1, page_kvs) 35 | 36 | tree = Tree.add_page_to_tree(:gb_trees.empty(), page) 37 | 38 | # Should be able to find the page for keys in its range 39 | assert Tree.page_for_key(tree, "key1") == 1 40 | assert Tree.page_for_key(tree, "key2") == 1 41 | assert Tree.page_for_key(tree, "key3") == 1 42 | 43 | # In gap-free design, all keys map to a page 44 | # "key0" would go in page 1 45 | assert Tree.page_for_key(tree, "key0") == 1 46 | # "key4" beyond all pages goes to rightmost (0) 47 | assert Tree.page_for_key(tree, "key4") == 0 48 | end 49 | end 50 | end 51 | -------------------------------------------------------------------------------- /lib/bedrock/cluster/gateway/telemetry.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.Cluster.Gateway.Telemetry do 2 | @moduledoc false 3 | alias Bedrock.Cluster 4 | alias Bedrock.ControlPlane.Coordinator 5 | alias Bedrock.Telemetry 6 | 7 | @spec trace_started(cluster :: module()) :: :ok 8 | def trace_started(cluster) do 9 | Telemetry.execute([:bedrock, :cluster, :gateway, :started], %{}, %{ 10 | cluster: cluster 11 | }) 12 | end 13 | 14 | @spec trace_advertising_capabilities( 15 | cluster :: module(), 16 | capabilities :: [Cluster.capability()], 17 | running_services :: map() 18 | ) :: 19 | :ok 20 | def trace_advertising_capabilities(cluster, capabilities, running_services) do 21 | Telemetry.execute([:bedrock, :cluster, :gateway, :advertise_capabilities], %{}, %{ 22 | cluster: cluster, 23 | capabilities: capabilities, 24 | running_services: running_services 25 | }) 26 | end 27 | 28 | @spec trace_searching_for_coordinator(cluster :: module()) :: :ok 29 | def trace_searching_for_coordinator(cluster) do 30 | Telemetry.execute([:bedrock, :cluster, :gateway, :searching_for_coordinator], %{}, %{ 31 | cluster: cluster 32 | }) 33 | end 34 | 35 | @spec trace_found_coordinator( 36 | cluster :: module(), 37 | coordinator :: Coordinator.ref() 38 | ) :: :ok 39 | def trace_found_coordinator(cluster, coordinator) do 40 | Telemetry.execute([:bedrock, :cluster, :gateway, :found_coordinator], %{}, %{ 41 | cluster: cluster, 42 | coordinator: coordinator 43 | }) 44 | end 45 | 46 | @spec trace_missed_pong(cluster :: module(), n_missed :: non_neg_integer()) :: :ok 47 | def trace_missed_pong(cluster, n_missed) do 48 | Telemetry.execute([:bedrock, :cluster, :gateway, :missed_pong], %{missed_pongs: n_missed}, %{ 49 | cluster: cluster 50 | }) 51 | end 52 | end 53 | -------------------------------------------------------------------------------- /lib/bedrock/control_plane/director/state.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.ControlPlane.Director.State do 2 | @moduledoc false 3 | 4 | alias Bedrock.Cluster 5 | alias Bedrock.ControlPlane.Config 6 | alias Bedrock.ControlPlane.Config.TransactionSystemLayout 7 | alias Bedrock.Service.Worker 8 | 9 | @type state :: :starting | :recovery | :running | :stopped 10 | @type timer_registry :: %{atom() => reference()} 11 | 12 | @type t :: %__MODULE__{ 13 | state: state(), 14 | epoch: Bedrock.epoch(), 15 | cluster: module(), 16 | config: Config.t() | nil, 17 | transaction_system_layout: TransactionSystemLayout.t() | nil, 18 | old_transaction_system_layout: TransactionSystemLayout.t() | nil, 19 | coordinator: pid(), 20 | node_capabilities: %{Cluster.capability() => [node()]}, 21 | timers: timer_registry() | nil, 22 | services: %{Worker.id() => {atom(), {atom(), node()}}}, 23 | lock_token: binary(), 24 | recovery_attempt: Config.RecoveryAttempt.t() | nil 25 | } 26 | defstruct state: :starting, 27 | epoch: nil, 28 | cluster: nil, 29 | config: nil, 30 | transaction_system_layout: nil, 31 | old_transaction_system_layout: nil, 32 | coordinator: nil, 33 | node_capabilities: %{}, 34 | timers: nil, 35 | services: %{}, 36 | lock_token: nil, 37 | recovery_attempt: nil 38 | 39 | defmodule Changes do 40 | @moduledoc false 41 | 42 | alias Bedrock.ControlPlane.Director.State 43 | 44 | @spec put_state(State.t(), State.state()) :: State.t() 45 | def put_state(t, state), do: %{t | state: state} 46 | 47 | @spec update_config(State.t(), updater :: (Config.t() -> Config.t())) :: State.t() 48 | def update_config(t, updater), do: %{t | config: updater.(t.config)} 49 | end 50 | end 51 | -------------------------------------------------------------------------------- /test/bedrock/control_plane/coordinator/path_config_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.ControlPlane.Coordinator.PathConfigTest do 2 | use ExUnit.Case, async: false 3 | 4 | alias Bedrock.ControlPlane.Coordinator.DiskRaftLog 5 | alias Bedrock.ControlPlane.Coordinator.Server 6 | alias Bedrock.Raft.Log.TupleInMemoryLog 7 | 8 | @moduletag :tmp_dir 9 | 10 | # Mock cluster to test the path configuration pattern 11 | defmodule TestPathCluster do 12 | @moduledoc false 13 | def node_config do 14 | Application.get_env(:bedrock, __MODULE__, []) 15 | end 16 | end 17 | 18 | # Helper to configure cluster with coordinator path 19 | defp configure_cluster_with_path(coordinator_path) do 20 | Application.put_env(:bedrock, TestPathCluster, coordinator: [path: coordinator_path]) 21 | end 22 | 23 | # Helper to configure cluster without path 24 | defp configure_cluster_without_path do 25 | Application.put_env(:bedrock, TestPathCluster, coordinator: []) 26 | end 27 | 28 | describe "coordinator path configuration pattern" do 29 | test "coordinator creates disk log and files when path provided", %{tmp_dir: tmp_dir} do 30 | coordinator_path = Path.join(tmp_dir, "coordinator") 31 | configure_cluster_with_path(coordinator_path) 32 | 33 | # Verify it returns a DiskRaftLog and creates expected directory structure 34 | assert {:ok, %DiskRaftLog{} = raft_log} = Server.init_raft_log(TestPathCluster) 35 | 36 | working_directory = Path.join(coordinator_path, "raft") 37 | assert File.exists?(working_directory) 38 | assert File.exists?(Path.join(working_directory, "raft_log.dets")) 39 | 40 | DiskRaftLog.close(raft_log) 41 | end 42 | 43 | test "coordinator uses in-memory log when no path provided" do 44 | configure_cluster_without_path() 45 | 46 | assert {:ok, %TupleInMemoryLog{}} = Server.init_raft_log(TestPathCluster) 47 | end 48 | end 49 | end 50 | -------------------------------------------------------------------------------- /test/bedrock/integration/brand_new_system_initialization_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.Integration.BrandNewSystemInitializationTest do 2 | # Not async due to potential file system conflicts 3 | use ExUnit.Case, async: false 4 | 5 | alias Bedrock.ControlPlane.Director.Recovery.LogReplayPhase 6 | 7 | @moduletag :integration 8 | 9 | describe "copy_log_data/5" do 10 | # Helper function to attempt log data copy and handle expected failures 11 | defp attempt_copy_log_data(new_log_id, old_log_id, first_version, last_version, service_pids) do 12 | LogReplayPhase.copy_log_data(new_log_id, old_log_id, first_version, last_version, service_pids) 13 | catch 14 | # Expected failures when using mock process (self()) 15 | :exit, {:noproc, _} -> :attempted_log_call 16 | :exit, {:calling_self, {GenServer, :call, [_, {:recover_from, nil, 0, 0}, _]}} -> :new_system_call 17 | :exit, {:calling_self, {GenServer, :call, [_, {:recover_from, _, _, _}, _]}} -> :existing_system_call 18 | :exit, {reason, _} when reason in [:normal, :killed, :shutdown] -> :attempted_log_call 19 | error_type, reason -> {error_type, reason} 20 | end 21 | 22 | test "calls recover_from with nil source for brand new system" do 23 | # Verifies fix: :none old_log_id calls Log.recover_from instead of returning early 24 | service_pids = %{"test_log" => self()} 25 | 26 | # Brand new system should call recover_from with nil source and version 0 27 | assert :new_system_call = attempt_copy_log_data("test_log", :none, 0, 0, service_pids) 28 | end 29 | 30 | test "calls recover_from with source log for existing system recovery" do 31 | service_pids = %{"new_log" => self(), "old_log" => self()} 32 | 33 | # Existing system should call recover_from with source log reference 34 | assert :existing_system_call = attempt_copy_log_data("new_log", "old_log", 10, 20, service_pids) 35 | end 36 | end 37 | end 38 | -------------------------------------------------------------------------------- /test/bedrock/control_plane/coordinator/end_epoch_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.ControlPlane.Coordinator.EndEpochTest do 2 | use ExUnit.Case, async: true 3 | 4 | alias Bedrock.ControlPlane.Coordinator.Durability 5 | alias Bedrock.ControlPlane.Coordinator.State 6 | 7 | # Helper function to create base state with overrides 8 | defp create_state(overrides) do 9 | defaults = [ 10 | cluster: :test_cluster, 11 | director: :unavailable, 12 | leader_node: Node.self(), 13 | my_node: Node.self(), 14 | supervisor_otp_name: nil 15 | ] 16 | 17 | struct!(State, Keyword.merge(defaults, overrides)) 18 | end 19 | 20 | describe "end_epoch command processing" do 21 | test "end_epoch command shuts down director when leader with running director" do 22 | director_pid = spawn(fn -> :timer.sleep(100) end) 23 | {:ok, test_supervisor} = DynamicSupervisor.start_link(strategy: :one_for_one) 24 | 25 | state = create_state(director: director_pid, supervisor_otp_name: test_supervisor) 26 | 27 | # Should set director to :unavailable when processing end_epoch 28 | assert %State{director: :unavailable} = 29 | Durability.process_command(state, {:end_epoch, 42}) 30 | 31 | DynamicSupervisor.stop(test_supervisor) 32 | end 33 | 34 | test "end_epoch command does nothing when not leader" do 35 | director_pid = spawn(fn -> :timer.sleep(100) end) 36 | state = create_state(director: director_pid, leader_node: :other_node) 37 | 38 | # Should leave state unchanged when not leader 39 | assert ^state = Durability.process_command(state, {:end_epoch, 42}) 40 | end 41 | 42 | test "end_epoch command does nothing when no director running" do 43 | state = create_state(director: :unavailable) 44 | 45 | # Should leave state unchanged when no director is running 46 | assert ^state = Durability.process_command(state, {:end_epoch, 42}) 47 | end 48 | end 49 | end 50 | -------------------------------------------------------------------------------- /lib/bedrock/control_plane/coordinator/raft_adapter.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.ControlPlane.Coordinator.RaftAdapter do 2 | @moduledoc false 3 | @behaviour Bedrock.Raft.Interface 4 | 5 | require Logger 6 | 7 | @spec determine_timeout(pos_integer(), pos_integer()) :: pos_integer() 8 | defp determine_timeout(min_ms, max_ms) when min_ms == max_ms, do: min_ms 9 | defp determine_timeout(min_ms, max_ms) when min_ms > max_ms, do: raise("invalid_timeout") 10 | defp determine_timeout(min_ms, max_ms), do: min_ms + :rand.uniform(max_ms - min_ms) 11 | 12 | @impl true 13 | def heartbeat_ms, do: 100 14 | 15 | @impl true 16 | def timestamp_in_ms, do: :erlang.monotonic_time(:millisecond) 17 | 18 | @impl true 19 | def ignored_event(_event, _from), do: :ok 20 | 21 | @impl true 22 | def leadership_changed(leadership), do: send(self(), {:raft, :leadership_changed, leadership}) 23 | 24 | @impl true 25 | def send_event(to, event) do 26 | send(self(), {:raft, :send_rpc, event, to}) 27 | :ok 28 | end 29 | 30 | @impl true 31 | def timer(:heartbeat), do: set_timer(:heartbeat, heartbeat_ms(), 0) 32 | def timer(:election), do: set_timer(:election, 250, 100) 33 | 34 | @spec set_timer(atom(), pos_integer(), non_neg_integer()) :: (-> :ok | {:error, :badarg}) 35 | defp set_timer(name, min_ms, jitter) do 36 | min_ms 37 | |> determine_timeout(min_ms + jitter) 38 | |> :timer.send_after({:raft, :timer, name}) 39 | |> case do 40 | {:ok, ref} -> 41 | fn -> :timer.cancel(ref) end 42 | 43 | {:error, _} -> 44 | raise "Bedrock: failed to start timer for raft #{inspect(name)}" 45 | 46 | fn -> :ok end 47 | end 48 | end 49 | 50 | @impl true 51 | def consensus_reached(log, transaction_id, consistency) do 52 | send(self(), {:raft, :consensus_reached, log, transaction_id, consistency}) 53 | :ok 54 | end 55 | 56 | @impl true 57 | def quorum_lost(_active_followers, _total_followers, _term), do: :step_down 58 | end 59 | -------------------------------------------------------------------------------- /test/support/storage/olivine/test_helpers.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.Storage.Olivine.TestHelpers do 2 | @moduledoc """ 3 | Test helpers for Olivine storage testing. 4 | """ 5 | 6 | alias Bedrock.DataPlane.Storage.Olivine.IndexDatabase 7 | alias Bedrock.DataPlane.Version 8 | 9 | @doc """ 10 | Load version range metadata only (without loading all pages into memory). 11 | Returns version ranges with empty page maps for testing purposes. 12 | """ 13 | @spec load_version_range_metadata(IndexDatabase.t()) :: [{Bedrock.version(), Bedrock.version()}] 14 | def load_version_range_metadata(index_db) do 15 | # Scan the append-only file to extract version metadata 16 | index_db.file 17 | |> scan_all_version_metadata(index_db.file_offset, []) 18 | |> Enum.sort_by(fn {version, _last_version} -> version end, &(Version.compare(&1, &2) != :lt)) 19 | end 20 | 21 | # Constants matching IndexDatabase 22 | @magic_number 0x4F4C5644 23 | @header_size 16 24 | @footer_size 4 25 | @min_record_size @header_size + @footer_size 26 | 27 | defp scan_all_version_metadata(_file, current_offset, acc) when current_offset < @min_record_size, 28 | do: Enum.reverse(acc) 29 | 30 | defp scan_all_version_metadata(file, current_offset, acc) do 31 | with {:ok, <>} <- :file.pread(file, current_offset - @footer_size, @footer_size), 32 | record_size = @header_size + payload_size + @footer_size, 33 | record_offset = current_offset - record_size, 34 | true <- record_offset >= 0, 35 | {:ok, 36 | <<@magic_number::32, version::binary-size(8), ^payload_size::32, payload::binary-size(payload_size), 37 | ^payload_size::32>>} <- :file.pread(file, record_offset, record_size) do 38 | {previous_version, _pages_map} = :erlang.binary_to_term(payload) 39 | scan_all_version_metadata(file, record_offset, [{version, previous_version} | acc]) 40 | else 41 | _ -> Enum.reverse(acc) 42 | end 43 | end 44 | end 45 | -------------------------------------------------------------------------------- /test/bedrock/directory/metadata_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.Directory.MetadataTest do 2 | use ExUnit.Case, async: true 3 | use ExUnitProperties 4 | 5 | import Bedrock.Test.DirectoryHelpers 6 | import Mox 7 | import StreamData 8 | 9 | alias Bedrock.Directory 10 | 11 | setup do 12 | stub(MockRepo, :transact, fn callback -> callback.() end) 13 | :ok 14 | end 15 | 16 | setup :verify_on_exit! 17 | 18 | defp non_reserved_prefix do 19 | [min_length: 1, max_length: 8] 20 | |> binary() 21 | |> filter(fn prefix -> 22 | not String.starts_with?(prefix, <<0xFE>>) and not String.starts_with?(prefix, <<0xFF>>) 23 | end) 24 | end 25 | 26 | describe "directory metadata preservation" do 27 | property "create and open non-root directory with manual prefix preserves all metadata" do 28 | check all( 29 | path_name <- string(:alphanumeric, min_length: 1, max_length: 10), 30 | prefix <- non_reserved_prefix() 31 | ) do 32 | expect_version_initialization(MockRepo) 33 | layer = Directory.root(MockRepo, next_prefix_fn: fn -> prefix end) 34 | 35 | path = [path_name] 36 | layer_name = nil 37 | packed_value = pack_directory_value(prefix, "") 38 | 39 | MockRepo 40 | |> expect_directory_exists(path, nil) 41 | |> expect_collision_check(prefix) 42 | |> expect_ancestor_checks(byte_size(prefix) - 1) 43 | |> expect_directory_creation(path, {prefix, ""}) 44 | 45 | assert {:ok, 46 | %{ 47 | prefix: ^prefix, 48 | layer: ^layer_name, 49 | path: ^path 50 | } = created_node} = 51 | Directory.create(layer, path, layer: layer_name, prefix: prefix) 52 | 53 | MockRepo 54 | |> expect_directory_exists(path, packed_value) 55 | |> expect_version_check() 56 | 57 | assert {:ok, ^created_node} = Directory.open(layer, path) 58 | end 59 | end 60 | end 61 | end 62 | -------------------------------------------------------------------------------- /test/support/gateway/server_test_support.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.Test.Gateway.ServerTestSupport do 2 | @moduledoc """ 3 | Test support utilities for Bedrock.Cluster.Gateway.Server tests. 4 | Provides common state creation and setup functions. 5 | """ 6 | 7 | alias Bedrock.Cluster.Descriptor 8 | alias Bedrock.Cluster.Gateway.State 9 | 10 | # Test cluster module used across server tests 11 | defmodule TestCluster do 12 | @moduledoc false 13 | def name, do: "test_cluster" 14 | def otp_name(:coordinator), do: :test_coordinator 15 | def otp_name(component), do: :"test_#{component}" 16 | def coordinator_ping_timeout_in_ms, do: 5000 17 | def coordinator_discovery_timeout_in_ms, do: 5000 18 | def gateway_ping_timeout_in_ms, do: 5000 19 | def capabilities, do: [:storage, :log] 20 | end 21 | 22 | @doc """ 23 | Creates a base state for testing with optional overrides. 24 | """ 25 | def base_state(overrides \\ []) do 26 | defaults = %State{ 27 | node: :test_node, 28 | cluster: TestCluster, 29 | descriptor: %Descriptor{coordinator_nodes: [:node1, :node2]}, 30 | path_to_descriptor: "/test/path", 31 | known_coordinator: :test_coordinator, 32 | transaction_system_layout: nil, 33 | mode: :active, 34 | capabilities: [:storage, :log] 35 | } 36 | 37 | Map.merge(defaults, Map.new(overrides)) 38 | end 39 | 40 | @doc """ 41 | Creates valid child_spec options with optional overrides. 42 | """ 43 | def valid_child_spec_opts(overrides \\ []) do 44 | defaults = [ 45 | cluster: TestCluster, 46 | descriptor: %Descriptor{coordinator_nodes: [:node1, :node2]}, 47 | path_to_descriptor: "/path/to/descriptor", 48 | otp_name: :test_gateway, 49 | capabilities: [:storage, :log] 50 | ] 51 | 52 | Keyword.merge(defaults, overrides) 53 | end 54 | 55 | @doc """ 56 | Returns list of required child_spec options. 57 | """ 58 | def required_child_spec_options do 59 | [:cluster, :descriptor, :path_to_descriptor, :otp_name, :capabilities] 60 | end 61 | end 62 | -------------------------------------------------------------------------------- /test/support/genserver_test_helpers_demo_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.Test.GenServerTestHelpersDemoTest do 2 | @moduledoc """ 3 | Demonstration tests showing how to use the GenServer test helper macros. 4 | These tests show the proper patterns for testing GenServer calls and casts 5 | with pattern matching and timeout handling. 6 | """ 7 | 8 | use ExUnit.Case, async: true 9 | 10 | import Bedrock.Test.Common.GenServerTestHelpers 11 | 12 | # Helpers to simulate message sending in separate processes 13 | defp send_cast_message(test_pid, message) do 14 | spawn(fn -> GenServer.cast(test_pid, message) end) 15 | end 16 | 17 | defp send_call_message(test_pid, message) do 18 | spawn(fn -> send(test_pid, {:"$gen_call", {self(), make_ref()}, message}) end) 19 | end 20 | 21 | describe "GenServer test helper macros" do 22 | setup do 23 | [test_pid: self()] 24 | end 25 | 26 | test "assert_cast_received demonstrates cast message testing", %{test_pid: test_pid} do 27 | send_cast_message(test_pid, {:worker_health, "worker_1", {:ok, self()}}) 28 | 29 | # Use pattern matching to assert on exact message structure 30 | assert_cast_received({:worker_health, "worker_1", {:ok, pid}}) do 31 | assert is_pid(pid) 32 | end 33 | end 34 | 35 | test "assert_call_received with timeout demonstrates call message testing", %{test_pid: test_pid} do 36 | send_call_message(test_pid, {:recover_from, :log_1, 100, 200}) 37 | 38 | # Use pattern matching to assert on exact message structure with timeout 39 | assert_call_received({:recover_from, :log_1, 100, 200}, 200) 40 | end 41 | 42 | test "demonstrates simple message pattern matching", %{test_pid: test_pid} do 43 | # Test both cast and call message patterns 44 | send_cast_message(test_pid, {:notification, "test_message"}) 45 | assert_cast_received({:notification, "test_message"}) 46 | 47 | send_call_message(test_pid, {:fetch_data, :user_id_123}) 48 | assert_call_received({:fetch_data, :user_id_123}) 49 | end 50 | end 51 | end 52 | -------------------------------------------------------------------------------- /lib/bedrock/cluster/gateway/transaction_builder/finalization.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.Cluster.Gateway.TransactionBuilder.Finalization do 2 | @moduledoc false 3 | 4 | alias Bedrock.Cluster.Gateway.TransactionBuilder.State 5 | alias Bedrock.Cluster.Gateway.TransactionBuilder.Tx 6 | alias Bedrock.DataPlane.CommitProxy 7 | alias Bedrock.DataPlane.Transaction 8 | 9 | @spec commit(State.t()) :: {:ok, State.t()} | {:error, term()} 10 | @spec commit(State.t(), opts :: keyword()) :: {:ok, State.t()} | {:error, term()} 11 | def commit(t, opts \\ []) 12 | 13 | def commit(%{stack: [], tx: %{mutations: []}} = t, _opts) do 14 | commit_version = t.read_version || 0 15 | {:ok, %{t | state: :committed, commit_version: commit_version}} 16 | end 17 | 18 | def commit(%{stack: []} = t, opts) do 19 | commit_fn = Keyword.get(opts, :commit_fn, &CommitProxy.commit/2) 20 | transaction = prepare_transaction_for_commit(t.read_version, t.tx) 21 | 22 | with {:ok, commit_proxy} <- select_commit_proxy(t.transaction_system_layout), 23 | {:ok, version, _sequence} <- commit_fn.(commit_proxy, transaction) do 24 | {:ok, %{t | state: :committed, commit_version: version}} 25 | end 26 | end 27 | 28 | def commit(%{stack: [_tx | stack]} = t, _opts), do: {:ok, %{t | stack: stack}} 29 | 30 | @spec prepare_transaction_for_commit( 31 | read_version :: Bedrock.version() | nil, 32 | tx :: Tx.t() 33 | ) :: 34 | Transaction.encoded() 35 | defp prepare_transaction_for_commit(read_version, tx) do 36 | Tx.commit(tx, read_version) 37 | end 38 | 39 | @spec select_commit_proxy(Bedrock.ControlPlane.Config.TransactionSystemLayout.t()) :: 40 | {:ok, CommitProxy.ref()} | {:error, :unavailable} 41 | defp select_commit_proxy(%{proxies: []}), do: {:error, :unavailable} 42 | defp select_commit_proxy(%{proxies: proxies}), do: {:ok, Enum.random(proxies)} 43 | 44 | @spec rollback(State.t()) :: :stop | State.t() 45 | def rollback(%{stack: []}), do: :stop 46 | def rollback(%{stack: [tx | stack]} = t), do: %{t | tx: tx, stack: stack} 47 | end 48 | -------------------------------------------------------------------------------- /lib/bedrock/data_plane/sequencer.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.Sequencer do 2 | @moduledoc """ 3 | Global version authority implementing Lamport clock semantics for MVCC transactions. 4 | 5 | The Sequencer assigns monotonically increasing version numbers that establish 6 | global ordering for transactions. Read versions provide consistent snapshot points 7 | while commit versions establish transaction ordering for conflict detection. 8 | 9 | Each commit version assignment updates the Lamport clock, ensuring the version 10 | pair returned maintains proper causality relationships needed for distributed 11 | MVCC conflict resolution. 12 | 13 | For detailed version assignment concepts and architectural integration, see the 14 | [Sequencer documentation](../../../../docs/components/sequencer.md). 15 | """ 16 | 17 | use Bedrock.Internal.GenServerApi, for: __MODULE__.Server 18 | 19 | @type ref :: pid() | atom() | {atom(), node()} 20 | 21 | @spec next_read_version( 22 | ref(), 23 | opts :: [ 24 | timeout_in_ms: Bedrock.timeout_in_ms() 25 | ] 26 | ) :: {:ok, Bedrock.version()} | {:error, :unavailable} 27 | def next_read_version(t, opts \\ []), do: call(t, :next_read_version, opts[:timeout_in_ms] || :infinity) 28 | 29 | @spec next_commit_version( 30 | ref(), 31 | opts :: [ 32 | timeout_in_ms: Bedrock.timeout_in_ms() 33 | ] 34 | ) :: 35 | {:ok, last_commit_version :: Bedrock.version(), next_commit_version :: Bedrock.version()} 36 | | {:error, :unavailable} 37 | def next_commit_version(t, opts \\ []), do: call(t, :next_commit_version, opts[:timeout_in_ms] || :infinity) 38 | 39 | @spec report_successful_commit( 40 | ref(), 41 | commit_version :: Bedrock.version(), 42 | opts :: [timeout_in_ms: Bedrock.timeout_in_ms()] 43 | ) :: :ok | {:error, :unavailable} 44 | def report_successful_commit(t, commit_version, opts \\ []), 45 | do: call(t, {:report_successful_commit, commit_version}, opts[:timeout_in_ms] || :infinity) 46 | end 47 | -------------------------------------------------------------------------------- /test/bedrock/data_plane/log/shale/facts_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.Log.Shale.FactsTest do 2 | use ExUnit.Case, async: true 3 | 4 | alias Bedrock.DataPlane.Log.Shale.Facts 5 | alias Bedrock.DataPlane.Log.Shale.State 6 | 7 | @test_state %State{ 8 | id: "test_id", 9 | otp_name: :test_otp, 10 | oldest_version: 1, 11 | last_version: 10 12 | } 13 | 14 | describe "info/2" do 15 | test "returns info for static facts" do 16 | assert {:ok, "test_id"} = Facts.info(@test_state, :id) 17 | assert {:ok, :log} = Facts.info(@test_state, :kind) 18 | assert {:ok, :test_otp} = Facts.info(@test_state, :otp_name) 19 | assert {:ok, :unavailable} = Facts.info(@test_state, :minimum_durable_version) 20 | assert {:ok, 1} = Facts.info(@test_state, :oldest_version) 21 | assert {:ok, 10} = Facts.info(@test_state, :last_version) 22 | end 23 | 24 | test "returns info for dynamic facts" do 25 | current_pid = self() 26 | supported_info = Facts.supported_info() 27 | 28 | assert {:ok, ^current_pid} = Facts.info(@test_state, :pid) 29 | assert {:ok, ^supported_info} = Facts.info(@test_state, :supported_info) 30 | end 31 | 32 | test "returns info for a list of facts" do 33 | facts = [:id, :kind, :otp_name] 34 | 35 | expected = %{ 36 | id: "test_id", 37 | kind: :log, 38 | otp_name: :test_otp 39 | } 40 | 41 | assert {:ok, ^expected} = Facts.info(@test_state, facts) 42 | end 43 | 44 | test "returns error for unsupported fact" do 45 | assert {:error, :unsupported} = Facts.info(@test_state, :unsupported_fact) 46 | end 47 | end 48 | 49 | describe "supported_info/0" do 50 | test "returns the list of supported info" do 51 | expected = [ 52 | :id, 53 | :kind, 54 | :minimum_durable_version, 55 | :oldest_version, 56 | :last_version, 57 | :otp_name, 58 | :pid, 59 | :state, 60 | :supported_info 61 | ] 62 | 63 | assert expected == Facts.supported_info() 64 | end 65 | end 66 | end 67 | -------------------------------------------------------------------------------- /lib/bedrock/service/foreman/state.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.Service.Foreman.State do 2 | @moduledoc false 3 | alias Bedrock.Cluster 4 | alias Bedrock.Service.Foreman.State 5 | alias Bedrock.Service.Foreman.WorkerInfo 6 | alias Bedrock.Service.Worker 7 | 8 | @type t :: %__MODULE__{ 9 | cluster: Cluster.t(), 10 | capabilities: [Cluster.capability()], 11 | health: :starting | :ok | :error, 12 | otp_name: atom(), 13 | path: Path.t(), 14 | waiting_for_healthy: [pid()], 15 | workers: %{Worker.id() => WorkerInfo.t()} 16 | } 17 | defstruct [ 18 | :cluster, 19 | :capabilities, 20 | :default_worker, 21 | :health, 22 | :otp_name, 23 | :path, 24 | :waiting_for_healthy, 25 | :workers 26 | ] 27 | 28 | def new_state(%{cluster: cluster, capabilities: capabilities, path: path, otp_name: otp_name}) do 29 | {:ok, 30 | %__MODULE__{ 31 | cluster: cluster, 32 | capabilities: capabilities, 33 | path: path, 34 | otp_name: otp_name, 35 | # 36 | health: :starting, 37 | waiting_for_healthy: [], 38 | workers: %{} 39 | }} 40 | end 41 | 42 | def new_state(_), do: {:error, :missing_required_params} 43 | 44 | def update_workers(t, updater), do: %{t | workers: updater.(t.workers)} 45 | 46 | def update_health(t, updater), do: %{t | health: updater.(t.health)} 47 | 48 | def put_health(t, health), do: %{t | health: health} 49 | 50 | @spec update_waiting_for_healthy(State.t(), ([GenServer.from()] -> [GenServer.from()])) :: 51 | State.t() 52 | def update_waiting_for_healthy(t, updater), do: %{t | waiting_for_healthy: updater.(t.waiting_for_healthy)} 53 | 54 | @spec put_waiting_for_healthy(State.t(), [pid()]) :: State.t() 55 | def put_waiting_for_healthy(t, waiting_for_healthy), do: %{t | waiting_for_healthy: waiting_for_healthy} 56 | 57 | @spec put_health_for_worker(State.t(), Worker.id(), Worker.health()) :: State.t() 58 | def put_health_for_worker(t, worker_id, health), 59 | do: update_workers(t, fn workers -> Map.update!(workers, worker_id, &WorkerInfo.put_health(&1, health)) end) 60 | end 61 | -------------------------------------------------------------------------------- /test/bedrock/control_plane/director/component_monitoring_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.ControlPlane.Director.ComponentMonitoringTest do 2 | use ExUnit.Case, async: true 3 | 4 | alias Bedrock.ControlPlane.Director.Server 5 | 6 | describe "component failure handling" do 7 | test "terminates with shutdown reason when component fails" do 8 | # Spawn a test director process 9 | test_process = self() 10 | 11 | director_pid = 12 | spawn(fn -> 13 | # Simulate director receiving :DOWN message 14 | send(test_process, {:director_started, self()}) 15 | 16 | receive do 17 | {:simulate_component_failure, failed_pid, reason} -> 18 | # This should cause the director to exit 19 | send(self(), {:DOWN, make_ref(), :process, failed_pid, reason}) 20 | 21 | # Use the actual handle_info logic 22 | case Server.handle_info({:DOWN, make_ref(), :process, failed_pid, reason}, %{}) do 23 | {:stop, exit_reason, _state} -> 24 | send(test_process, {:director_exited, exit_reason}) 25 | exit(exit_reason) 26 | 27 | other -> 28 | send(test_process, {:unexpected_result, other}) 29 | exit(:unexpected_result) 30 | end 31 | end 32 | end) 33 | 34 | # Wait for director to start 35 | assert_receive {:director_started, ^director_pid} 36 | 37 | # Monitor the director 38 | monitor_ref = Process.monitor(director_pid) 39 | 40 | # Simulate component failure 41 | failed_component_pid = spawn(fn -> :ok end) 42 | failure_reason = :test_failure 43 | expected_shutdown_reason = {:shutdown, {:component_failure, failed_component_pid, failure_reason}} 44 | 45 | send(director_pid, {:simulate_component_failure, failed_component_pid, failure_reason}) 46 | 47 | # Director should exit immediately with proper shutdown reason 48 | assert_receive {:director_exited, ^expected_shutdown_reason} 49 | assert_receive {:DOWN, ^monitor_ref, :process, ^director_pid, ^expected_shutdown_reason} 50 | end 51 | end 52 | end 53 | -------------------------------------------------------------------------------- /lib/bedrock/data_plane/storage/olivine/state.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.Storage.Olivine.State do 2 | @moduledoc false 3 | 4 | alias Bedrock.ControlPlane.Director 5 | alias Bedrock.DataPlane.Storage.Olivine.Database 6 | alias Bedrock.DataPlane.Storage.Olivine.IndexManager 7 | alias Bedrock.DataPlane.Storage.Olivine.IntakeQueue 8 | alias Bedrock.DataPlane.Storage.Olivine.Reading 9 | alias Bedrock.Service.Foreman 10 | alias Bedrock.Service.Worker 11 | 12 | @type t :: %__MODULE__{ 13 | otp_name: atom(), 14 | path: Path.t(), 15 | foreman: Foreman.ref(), 16 | id: Worker.id(), 17 | database: Database.t(), 18 | index_manager: IndexManager.t(), 19 | pull_task: Task.t() | nil, 20 | epoch: Bedrock.epoch() | nil, 21 | director: Director.ref() | nil, 22 | mode: :locked | :running, 23 | read_request_manager: Reading.t(), 24 | intake_queue: IntakeQueue.t(), 25 | window_lag_time_μs: non_neg_integer(), 26 | compaction_task: Task.t() | nil, 27 | allow_window_advancement: boolean() 28 | } 29 | defstruct otp_name: nil, 30 | path: nil, 31 | foreman: nil, 32 | id: nil, 33 | database: nil, 34 | index_manager: nil, 35 | pull_task: nil, 36 | epoch: nil, 37 | director: nil, 38 | mode: :locked, 39 | read_request_manager: Reading.new(), 40 | intake_queue: IntakeQueue.new(), 41 | window_lag_time_μs: 5_000_000, 42 | compaction_task: nil, 43 | allow_window_advancement: true 44 | 45 | @spec update_mode(t(), :locked | :running) :: t() 46 | def update_mode(t, mode), do: %{t | mode: mode} 47 | 48 | @spec update_director_and_epoch(t(), Director.ref() | nil, Bedrock.epoch() | nil) :: t() 49 | def update_director_and_epoch(t, director, epoch), do: %{t | director: director, epoch: epoch} 50 | 51 | @spec reset_puller(t()) :: t() 52 | def reset_puller(t), do: %{t | pull_task: nil} 53 | 54 | @spec put_puller(t(), Task.t()) :: t() 55 | def put_puller(t, pull_task), do: %{t | pull_task: pull_task} 56 | end 57 | -------------------------------------------------------------------------------- /test/test_helper.exs: -------------------------------------------------------------------------------- 1 | ExUnit.start() 2 | Faker.start() 3 | 4 | # Removed global telemetry handler - tests should set up their own handlers as needed 5 | 6 | # Default test cluster for telemetry 7 | defmodule DefaultTestCluster do 8 | @moduledoc false 9 | def name, do: "test_cluster" 10 | def otp_name(component), do: :"test_#{component}" 11 | 12 | @doc """ 13 | Sets up logger metadata for telemetry handlers. 14 | Call this in test setup to ensure proper cluster identification. 15 | """ 16 | def setup_telemetry_metadata(epoch \\ 1) do 17 | Logger.metadata(cluster: __MODULE__, epoch: epoch) 18 | :ok 19 | end 20 | end 21 | 22 | Mox.defmock(Bedrock.Raft.MockInterface, for: Bedrock.Raft.Interface) 23 | Mox.defmock(MockRepo, for: Bedrock.Repo) 24 | Mox.stub(MockRepo, :transact, fn callback -> callback.() end) 25 | 26 | # Define behavior for Resolver testing 27 | defmodule Bedrock.DataPlane.Resolver.Behaviour do 28 | @moduledoc false 29 | @callback resolve_transactions( 30 | ref :: any(), 31 | last_version :: Bedrock.version(), 32 | commit_version :: Bedrock.version(), 33 | [Bedrock.DataPlane.Resolver.transaction()], 34 | keyword() 35 | ) :: 36 | {:ok, aborted :: [index :: integer()]} 37 | | {:error, :timeout} 38 | | {:error, :unavailable} 39 | end 40 | 41 | Mox.defmock(Bedrock.DataPlane.ResolverMock, for: Bedrock.DataPlane.Resolver.Behaviour) 42 | 43 | defmodule QuietDetsHandler do 44 | @moduledoc false 45 | def init(_) do 46 | {:ok, []} 47 | end 48 | 49 | def handle_event({:error_report, _gl, {_pid, :dets, _report}}, state) do 50 | # Silently ignore DETS error reports 51 | {:ok, state} 52 | end 53 | 54 | def handle_event(event, state) do 55 | # Let other events through to default handler 56 | :error_logger_tty_h.handle_event(event, state) 57 | end 58 | 59 | # Other required callbacks... 60 | def handle_call(_request, state), do: {:ok, :ok, state} 61 | def handle_info(_info, state), do: {:ok, state} 62 | def terminate(_reason, _state), do: :ok 63 | end 64 | 65 | # Add the handler 66 | :error_logger.add_report_handler(QuietDetsHandler, []) 67 | -------------------------------------------------------------------------------- /lib/bedrock/control_plane/director/recovery/monitoring_phase.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.ControlPlane.Director.Recovery.MonitoringPhase do 2 | @moduledoc """ 3 | Sets up monitoring of all transaction system components and marks recovery as complete. 4 | 5 | Establishes process monitoring for sequencer, commit proxies, resolvers, logs, 6 | and storage servers. Any failure of these critical components will trigger 7 | immediate director shutdown and recovery restart. 8 | 9 | This monitoring implements Bedrock's fail-fast philosophy - rather than 10 | attempting complex error recovery, component failures cause the director 11 | to exit and let the coordinator restart recovery with a new epoch. 12 | 13 | The monitoring setup represents the final step before the cluster becomes 14 | operational. Once monitoring is active, the director shifts from recovery 15 | mode to operational mode. 16 | """ 17 | 18 | use Bedrock.ControlPlane.Director.Recovery.RecoveryPhase 19 | 20 | import Bedrock.ControlPlane.Director.Recovery.Telemetry 21 | 22 | require Logger 23 | 24 | @impl true 25 | def execute(recovery_attempt, context) do 26 | trace_recovery_monitoring_components() 27 | 28 | monitor_fn = Map.get(context, :monitor_fn, &Process.monitor/1) 29 | 30 | recovery_attempt.transaction_system_layout 31 | |> extract_pids_to_monitor() 32 | |> monitor_all_pids(monitor_fn) 33 | 34 | {recovery_attempt, Bedrock.ControlPlane.Director.Recovery.PersistencePhase} 35 | end 36 | 37 | @spec extract_pids_to_monitor(map()) :: [pid()] 38 | defp extract_pids_to_monitor(layout) do 39 | resolver_pids = 40 | Enum.map(layout.resolvers, fn {_start_key, pid} -> pid end) 41 | 42 | service_pids = 43 | layout.services 44 | |> Enum.filter(fn {_service_id, service} -> service.kind != :storage end) 45 | |> Enum.map(fn {_service_id, %{status: {:up, pid}}} -> pid end) 46 | 47 | Enum.concat([ 48 | [layout.sequencer], 49 | layout.proxies, 50 | resolver_pids, 51 | service_pids 52 | ]) 53 | end 54 | 55 | @spec monitor_all_pids([pid()], (pid() -> reference())) :: [pid()] 56 | defp monitor_all_pids(pids, monitor_fn) do 57 | Enum.each(pids, monitor_fn) 58 | pids 59 | end 60 | end 61 | -------------------------------------------------------------------------------- /mix.exs: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.MixProject do 2 | use Mix.Project 3 | 4 | def project do 5 | [ 6 | app: :bedrock, 7 | version: "0.3.1", 8 | elixir: "~> 1.15", 9 | start_permanent: Mix.env() == :prod, 10 | deps: deps(), 11 | docs: &docs/0, 12 | test_coverage: [tool: ExCoveralls], 13 | preferred_cli_env: [ 14 | coveralls: :test, 15 | "coveralls.json": :test, 16 | dialyzer: :dev 17 | ], 18 | elixirc_paths: elixirc_paths(Mix.env()), 19 | dialyzer: dialyzer(), 20 | aliases: aliases() 21 | ] 22 | end 23 | 24 | defp aliases, do: [quality: ["format --check-formatted", "credo --strict", "dialyzer"]] 25 | 26 | defp dialyzer do 27 | [ 28 | plt_core_path: "priv/plts", 29 | plt_file: {:no_warn, "priv/plts/dialyzer.plt"}, 30 | plt_add_apps: [:ex_unit, :mix] 31 | ] 32 | end 33 | 34 | # Run "mix help compile.app" to learn about applications. 35 | def application do 36 | [ 37 | extra_applications: [:logger, :crypto] 38 | ] 39 | end 40 | 41 | # Run "mix help deps" to learn about dependencies. 42 | defp deps do 43 | add_deps_for_dev_and_test([ 44 | {:bedrock_raft, git: "https://github.com/jallum/bedrock_raft.git", tag: "0.9.6"}, 45 | {:jason, "~> 1.4"}, 46 | {:telemetry, "~> 1.2"} 47 | ]) 48 | end 49 | 50 | def add_deps_for_dev_and_test(deps) do 51 | deps ++ 52 | [ 53 | {:stream_data, "~> 1.1", only: :test}, 54 | {:credo, "~> 1.7", only: [:dev, :test], runtime: false}, 55 | {:dialyxir, "~> 1.4", only: [:dev, :test], runtime: false}, 56 | {:faker, "~> 0.17", only: :test}, 57 | {:mix_test_watch, "~> 1.0", only: [:dev, :test], runtime: false}, 58 | {:mox, "~> 1.1", only: :test}, 59 | {:excoveralls, "~> 0.18", only: :test}, 60 | {:benchee, "~> 1.3", only: :dev}, 61 | {:ex_doc, "~> 0.34", only: :dev, runtime: false, warn_if_outdated: true}, 62 | {:styler, "~> 1.0", only: [:dev, :test], runtime: false} 63 | ] 64 | end 65 | 66 | defp docs do 67 | [ 68 | main: "Bedrock", 69 | extras: ["README.md"] 70 | ] 71 | end 72 | 73 | defp elixirc_paths(:test), do: ["lib", "test/support"] 74 | defp elixirc_paths(_), do: ["lib"] 75 | end 76 | -------------------------------------------------------------------------------- /lib/bedrock/data_plane/commit_proxy/batch.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.CommitProxy.Batch do 2 | @moduledoc false 3 | 4 | alias Bedrock.DataPlane.Transaction 5 | 6 | @type reply_fn :: ({:ok, Bedrock.version(), index :: non_neg_integer()} | {:error, :abort} -> :ok) 7 | 8 | @type t :: %__MODULE__{ 9 | started_at: Bedrock.timestamp_in_ms(), 10 | finalized_at: Bedrock.timestamp_in_ms() | nil, 11 | last_commit_version: Bedrock.version(), 12 | commit_version: Bedrock.version(), 13 | n_transactions: non_neg_integer(), 14 | buffer: [{index :: non_neg_integer(), reply_fn(), Transaction.encoded(), Task.t() | nil}] 15 | } 16 | defstruct started_at: nil, 17 | finalized_at: nil, 18 | last_commit_version: nil, 19 | commit_version: nil, 20 | n_transactions: 0, 21 | buffer: [] 22 | 23 | @spec new_batch( 24 | Bedrock.timestamp_in_ms(), 25 | last_commit_version :: Bedrock.version(), 26 | commit_version :: Bedrock.version() 27 | ) :: t() 28 | def new_batch(started_at, last_commit_version, commit_version) do 29 | %__MODULE__{ 30 | started_at: started_at, 31 | last_commit_version: last_commit_version, 32 | commit_version: commit_version, 33 | n_transactions: 0, 34 | buffer: [] 35 | } 36 | end 37 | 38 | @spec transactions_in_order(t()) :: [ 39 | {index :: non_neg_integer(), reply_fn(), Transaction.encoded(), Task.t() | nil} 40 | ] 41 | def transactions_in_order(t), do: Enum.reverse(t.buffer) 42 | 43 | @spec all_callers(t()) :: [reply_fn()] 44 | def all_callers(t), do: Enum.map(t.buffer, &elem(&1, 1)) 45 | 46 | @spec add_transaction(t(), Transaction.encoded(), reply_fn(), Task.t() | nil) :: t() 47 | def add_transaction(t, transaction, reply_fn, task) when is_binary(transaction) do 48 | index = t.n_transactions 49 | %{t | buffer: [{index, reply_fn, transaction, task} | t.buffer], n_transactions: index + 1} 50 | end 51 | 52 | @spec transaction_count(t()) :: non_neg_integer() 53 | def transaction_count(t), do: t.n_transactions 54 | 55 | @spec set_finalized_at(t(), Bedrock.timestamp_in_ms()) :: t() 56 | def set_finalized_at(t, finalized_at), do: %{t | finalized_at: finalized_at} 57 | end 58 | -------------------------------------------------------------------------------- /lib/bedrock/data_plane/commit_proxy/telemetry.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.CommitProxy.Telemetry do 2 | @moduledoc false 3 | alias Bedrock.Telemetry 4 | 5 | @type telemetry_metadata :: %{optional(atom()) => term()} 6 | 7 | @spec trace_metadata() :: telemetry_metadata() 8 | def trace_metadata, do: Process.get(:trace_metadata, %{}) 9 | 10 | @spec trace_metadata(metadata :: telemetry_metadata()) :: telemetry_metadata() 11 | def trace_metadata(metadata), do: Process.put(:trace_metadata, Enum.into(metadata, trace_metadata())) 12 | 13 | @spec trace_commit_proxy_batch_started( 14 | commit_version :: Bedrock.version(), 15 | n_transactions :: non_neg_integer(), 16 | started_at :: Bedrock.timestamp_in_ms() 17 | ) :: :ok 18 | def trace_commit_proxy_batch_started(commit_version, n_transactions, started_at) do 19 | Telemetry.execute( 20 | [:bedrock, :data_plane, :commit_proxy, :start], 21 | %{n_transactions: n_transactions}, 22 | Map.merge(trace_metadata(), %{commit_version: commit_version, started_at: started_at}) 23 | ) 24 | end 25 | 26 | @spec trace_commit_proxy_batch_finished( 27 | commit_version :: Bedrock.version(), 28 | n_aborts :: non_neg_integer(), 29 | n_oks :: non_neg_integer(), 30 | duration_μs :: Bedrock.interval_in_us() 31 | ) :: :ok 32 | def trace_commit_proxy_batch_finished(commit_version, n_aborts, n_oks, duration_μs) do 33 | Telemetry.execute( 34 | [:bedrock, :data_plane, :commit_proxy, :stop], 35 | %{n_oks: n_oks, n_aborts: n_aborts, duration_μs: duration_μs}, 36 | Map.put(trace_metadata(), :commit_version, commit_version) 37 | ) 38 | end 39 | 40 | @spec trace_commit_proxy_batch_failed( 41 | batch :: Bedrock.DataPlane.CommitProxy.Batch.t(), 42 | reason :: any(), 43 | duration_μs :: Bedrock.interval_in_us() 44 | ) :: :ok 45 | def trace_commit_proxy_batch_failed(batch, reason, duration_μs) do 46 | Telemetry.execute( 47 | [:bedrock, :data_plane, :commit_proxy, :failed], 48 | %{ 49 | n_transactions: length(batch.buffer), 50 | duration_μs: duration_μs, 51 | commit_version: batch.commit_version 52 | }, 53 | Map.put(trace_metadata(), :reason, reason) 54 | ) 55 | end 56 | end 57 | -------------------------------------------------------------------------------- /test/support/storage/olivine/index_manager_test_helpers.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.Test.Storage.Olivine.IndexManagerTestHelpers do 2 | @moduledoc """ 3 | Test helper functions for IndexManager operations that are only used in tests. 4 | """ 5 | 6 | alias Bedrock.DataPlane.Storage.Olivine.IndexManager 7 | alias Bedrock.DataPlane.Version 8 | 9 | @doc """ 10 | Returns the last committed version for the index manager. 11 | This function is useful for test assertions. 12 | """ 13 | @spec last_committed_version(IndexManager.t()) :: Bedrock.version() 14 | def last_committed_version(index_manager), do: index_manager.current_version 15 | 16 | @doc """ 17 | Checks if a version is within the window (not evicted). 18 | This function is useful for test assertions about version visibility. 19 | """ 20 | @spec version_in_window?(Bedrock.version(), Bedrock.version()) :: boolean() 21 | def version_in_window?(version, window_start_version) do 22 | Version.compare(version, window_start_version) != :lt 23 | end 24 | 25 | @doc """ 26 | Splits versions list at the window boundary. 27 | Returns {versions_in_window, evicted_versions}. 28 | This function is useful for testing window advancement logic. 29 | """ 30 | @spec split_versions_at_window([{Bedrock.version(), term()}], Bedrock.version()) :: 31 | {[{Bedrock.version(), term()}], [{Bedrock.version(), term()}]} 32 | def split_versions_at_window(versions, window_start_version) do 33 | Enum.split_with(versions, fn {version, _index} -> 34 | version_in_window?(version, window_start_version) 35 | end) 36 | end 37 | 38 | @doc """ 39 | Finds the best index to use for a fetch at the given version. 40 | Returns the index with the highest version that is <= target_version. 41 | This function is useful for testing version resolution logic. 42 | """ 43 | @spec find_best_index_for_fetch([{Bedrock.version(), term()}], Bedrock.version()) :: 44 | {Bedrock.version(), term()} | nil 45 | def find_best_index_for_fetch([], _target_version), do: nil 46 | 47 | def find_best_index_for_fetch(versions, target_version) do 48 | versions 49 | |> Enum.filter(fn {version, _index} -> Version.compare(version, target_version) != :gt end) 50 | |> Enum.max_by(fn {version, _index} -> version end, fn -> nil end) 51 | end 52 | end 53 | -------------------------------------------------------------------------------- /lib/bedrock/cluster/gateway/tracing.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.Cluster.Gateway.Tracing do 2 | @moduledoc false 3 | 4 | require Logger 5 | 6 | @spec handler_id() :: String.t() 7 | defp handler_id, do: "bedrock_trace_gateway" 8 | 9 | @spec start() :: :ok | {:error, :already_exists} 10 | def start do 11 | :telemetry.attach_many( 12 | handler_id(), 13 | [ 14 | [:bedrock, :cluster, :gateway, :started], 15 | [:bedrock, :cluster, :gateway, :advertise_capabilities], 16 | [:bedrock, :cluster, :gateway, :searching_for_coordinator], 17 | [:bedrock, :cluster, :gateway, :found_coordinator] 18 | ], 19 | &__MODULE__.handler/4, 20 | nil 21 | ) 22 | end 23 | 24 | @spec stop() :: :ok | {:error, :not_found} 25 | def stop, do: :telemetry.detach(handler_id()) 26 | 27 | @spec handler( 28 | :telemetry.event_name(), 29 | :telemetry.event_measurements(), 30 | :telemetry.event_metadata(), 31 | any() 32 | ) :: any() 33 | def handler([:bedrock, :cluster, :gateway, event], measurements, metadata, _), 34 | do: trace(event, measurements, metadata) 35 | 36 | @spec trace(atom(), map(), map()) :: :ok 37 | def trace(:started, _, %{cluster: cluster}) do 38 | Logger.metadata(cluster: cluster) 39 | 40 | info("Gateway started") 41 | end 42 | 43 | def trace(:advertise_capabilities, _, %{capabilities: capabilities, running_services: running_services}) do 44 | info("Advertising capabilities (#{Enum.join(capabilities, ", ")}): #{inspect(running_services, pretty: true)}") 45 | end 46 | 47 | def trace(:searching_for_coordinator, _, _), do: info("Searching for a coordinator") 48 | 49 | def trace(:found_coordinator, _, %{coordinator: coordinator}), do: info("Found coordinator: #{inspect(coordinator)}") 50 | 51 | def trace(:missed_pong, %{missed_pongs: missed_pongs}, _) when missed_pongs > 1, 52 | do: info("Missed #{inspect(missed_pongs)} pongs from coordinator") 53 | 54 | def trace(:missed_pong, %{missed_pongs: missed_pongs}, _), 55 | do: info("Missed #{inspect(missed_pongs)} pong from coordinator") 56 | 57 | @spec info(message :: String.t()) :: :ok 58 | def info(message) do 59 | cluster = Keyword.fetch!(Logger.metadata(), :cluster) 60 | Logger.info("Bedrock [#{cluster.name()}]: #{message}") 61 | end 62 | end 63 | -------------------------------------------------------------------------------- /test/bedrock/control_plane/director/recovery/persistence_phase_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.ControlPlane.Director.Recovery.PersistencePhaseTest do 2 | use ExUnit.Case, async: true 3 | 4 | import Bedrock.Test.ControlPlane.RecoveryTestSupport 5 | 6 | alias Bedrock.ControlPlane.Director.Recovery.PersistencePhase 7 | 8 | # Shared test data setup 9 | defp mock_transaction_system_layout do 10 | %{ 11 | id: "test_layout_id", 12 | epoch: 1, 13 | director: self(), 14 | sequencer: self(), 15 | rate_keeper: nil, 16 | proxies: [self()], 17 | resolvers: [{<<0>>, self()}], 18 | logs: %{"log_1" => [1, 2]}, 19 | storage_teams: [], 20 | services: %{ 21 | "log_1" => %{status: {:up, self()}, kind: :log, last_seen: {:log_1, :node1}} 22 | } 23 | } 24 | end 25 | 26 | defp base_recovery_attempt do 27 | layout = mock_transaction_system_layout() 28 | 29 | recovery_attempt() 30 | |> with_sequencer(self()) 31 | |> with_proxies([self()]) 32 | |> with_resolvers([{<<0>>, self()}]) 33 | |> with_logs(%{"log_1" => [1, 2]}) 34 | |> with_transaction_services(%{ 35 | "log_1" => %{status: {:up, self()}, kind: :log, last_seen: {:log_1, :node1}} 36 | }) 37 | |> Map.put(:transaction_system_layout, layout) 38 | end 39 | 40 | describe "execute/2" do 41 | test "succeeds with existing transaction system layout and transitions to completed" do 42 | expected_layout = mock_transaction_system_layout() 43 | recovery_attempt = base_recovery_attempt() 44 | 45 | context = Map.put(recovery_context(), :commit_transaction_fn, fn _, _ -> {:ok, 1, 0} end) 46 | 47 | # Pattern match both result and next phase in single assertion 48 | assert {%{transaction_system_layout: ^expected_layout}, :completed} = 49 | PersistencePhase.execute(recovery_attempt, context) 50 | end 51 | 52 | test "fails when system transaction fails" do 53 | recovery_attempt = base_recovery_attempt() 54 | context = Map.put(recovery_context(), :commit_transaction_fn, fn _, _ -> {:error, :timeout} end) 55 | 56 | # Pattern match tuple destructuring with expected stall reason 57 | assert {_, {:stalled, {:recovery_system_failed, :timeout}}} = 58 | PersistencePhase.execute(recovery_attempt, context) 59 | end 60 | end 61 | end 62 | -------------------------------------------------------------------------------- /guides/deep-dives/architecture/control-plane/director.md: -------------------------------------------------------------------------------- 1 | # Director 2 | 3 | The [Director](../../../glossary.md#director) is Bedrock's [recovery](../../../glossary.md#recovery) orchestrator, serving as the singular authority for rebuilding the distributed [transaction](../../../glossary.md#transaction) system after failures. Created by the [Coordinator](../../../glossary.md#coordinator), it manages multi-phase service restoration, dependency sequencing, and ensures only one recovery process operates through [epoch](../../../glossary.md#epoch)-based authority. 4 | 5 | **Location**: [`lib/bedrock/control_plane/director.ex`](../../../lib/bedrock/control_plane/director.ex) 6 | 7 | ## Epoch-Based Authority 8 | 9 | Each Director receives a unique, monotonically increasing epoch number that determines recovery authority. Higher epochs win when multiple Directors exist, preventing split-brain scenarios. This eliminates complex distributed consensus during recovery when coordination infrastructure may be compromised. 10 | 11 | When Coordinator leadership changes during recovery, the new Coordinator creates a Director with a higher epoch, automatically superseding existing recovery processes. 12 | 13 | ## Recovery Orchestration 14 | 15 | Director executes multi-phase recovery addressing component dependencies. See the **[Overview](../../../quick-reads/recovery.md)** or the **[Deep Dive](../../../deep-dives/recovery.md)**. 16 | 17 | ## Service Lifecycle Management 18 | 19 | Director manages service creation and replacement through the use of local [Foreman](../../../glossary.md#foreman) on participating nodes. Placement decisions balance fault tolerance, network topology, and node capabilities to maximize reliability while minimizing recovery time. 20 | 21 | Health monitoring tracks functional capability of each component, ensuring services can perform their designated transaction system roles. 22 | 23 | ## Related Components 24 | 25 | - **[Coordinator](coordinator.md)**: Creates and manages Director lifecycle with epoch assignment 26 | - **[Foreman](../infrastructure/foreman.md)**: Infrastructure component that creates workers under Director coordination 27 | - **[Log](../data-plane/log.md)**: Critical data plane component requiring Director-managed recovery sequencing 28 | - **[Storage](../data-plane/storage.md)**: Data plane component with complex dependency requirements during recovery 29 | -------------------------------------------------------------------------------- /lib/bedrock.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock do 2 | @moduledoc """ 3 | Core types and utilities for Bedrock, a distributed key-value store. 4 | 5 | This module defines the fundamental types used throughout the Bedrock system, 6 | including keys, values, versions, and time-related constructs for MVCC 7 | transaction processing. 8 | """ 9 | 10 | alias Bedrock.Internal.Time.Interval 11 | 12 | @type key :: binary() 13 | @type key_range :: {min_inclusive :: key(), max_exclusive :: key() | :end} 14 | @type value :: binary() 15 | @type key_value :: {key(), value()} 16 | 17 | @type version :: Bedrock.DataPlane.Version.t() 18 | @type version_vector :: {oldest :: version(), newest :: version()} 19 | 20 | @type transaction :: %{ 21 | optional(:mutations) => [mutation()] | nil, 22 | optional(:write_conflicts) => [key_range()] | nil, 23 | optional(:read_conflicts) => {version(), [key_range()]} | nil, 24 | optional(:commit_version) => version() | nil 25 | } 26 | 27 | @type mutation :: 28 | {:set, key(), value()} 29 | | {:clear_range, key(), key()} 30 | 31 | @type epoch :: non_neg_integer() 32 | @type quorum :: pos_integer() 33 | @type timeout_in_ms :: :infinity | non_neg_integer() 34 | @type timestamp_in_ms :: integer() 35 | 36 | @type interval_in_ms :: :infinity | non_neg_integer() 37 | @type interval_in_us :: :infinity | non_neg_integer() 38 | 39 | @type time_unit :: Interval.unit() 40 | @type interval :: {Bedrock.time_unit(), non_neg_integer()} 41 | 42 | @type range_tag :: non_neg_integer() 43 | 44 | @type service :: :coordination | :log | :storage 45 | @type service_id :: String.t() 46 | @type lock_token :: binary() 47 | 48 | @doc """ 49 | Creates a key range from a minimum inclusive key to a maximum exclusive key. 50 | 51 | ## Parameters 52 | 53 | - `min_key`: The minimum key value (inclusive). 54 | - `max_key_exclusive`: The maximum key value (exclusive). 55 | 56 | ## Returns 57 | 58 | - A tuple representing the key range. 59 | 60 | ## Examples 61 | 62 | iex> Bedrock.key_range("a", "z") 63 | {"a", "z"} 64 | 65 | """ 66 | @spec key_range(Bedrock.key(), Bedrock.key() | :end) :: Bedrock.key_range() 67 | def key_range(min_key, max_key_exclusive) when min_key < max_key_exclusive or max_key_exclusive == :end, 68 | do: {min_key, max_key_exclusive} 69 | end 70 | -------------------------------------------------------------------------------- /lib/bedrock/control_plane/director/recovery/recovery_phase.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.ControlPlane.Director.Recovery.RecoveryPhase do 2 | @moduledoc """ 3 | Behavior for recovery phases in the Bedrock recovery process. 4 | 5 | All recovery phases should implement this behavior to ensure consistent 6 | interfaces and access to necessary context data. 7 | """ 8 | 9 | alias Bedrock.ControlPlane.Config 10 | alias Bedrock.ControlPlane.Config.RecoveryAttempt 11 | 12 | @type context :: %{ 13 | cluster_config: Config.t(), 14 | old_transaction_system_layout: Config.TransactionSystemLayout.t(), 15 | node_capabilities: %{Bedrock.Cluster.capability() => [node()]}, 16 | lock_token: binary(), 17 | available_services: %{String.t() => {atom(), {atom(), node()}}}, 18 | coordinator: pid() 19 | } 20 | 21 | @doc """ 22 | Execute the recovery phase with the given recovery attempt and context. 23 | 24 | The context provides access to Director state that phases need but 25 | shouldn't be stored in the recovery attempt itself. 26 | 27 | Returns: 28 | - `{updated_recovery_attempt, next_phase_module}` - Normal transition to next phase 29 | - `{updated_recovery_attempt, :completed}` - Terminal state, recovery complete 30 | - `updated_recovery_attempt` - Legacy terminal state for backward compatibility 31 | - `{recovery_attempt, {:stalled, reason}}` - Phase cannot proceed, will retry 32 | - `{recovery_attempt, :newer_epoch_exists}` - Director superseded, halt immediately 33 | 34 | **Stall vs Error Handling**: Stall conditions indicate temporary issues (insufficient 35 | resources, service unavailability) where recovery should retry. Error conditions like 36 | `:newer_epoch_exists` indicate permanent failures requiring immediate halt. 37 | """ 38 | @callback execute(RecoveryAttempt.t(), context()) :: 39 | {RecoveryAttempt.t(), module()} 40 | | {RecoveryAttempt.t(), :completed} 41 | | {RecoveryAttempt.t(), {:stalled, term()}} 42 | | {RecoveryAttempt.t(), :newer_epoch_exists} 43 | 44 | defmacro __using__(_) do 45 | quote do 46 | @behaviour Bedrock.ControlPlane.Director.Recovery.RecoveryPhase 47 | 48 | alias Bedrock.ControlPlane.Config.RecoveryAttempt 49 | alias Bedrock.ControlPlane.Director.Recovery.RecoveryPhase 50 | end 51 | end 52 | end 53 | -------------------------------------------------------------------------------- /guides/deep-dives/architecture/control-plane/coordinator.md: -------------------------------------------------------------------------------- 1 | # Coordinator 2 | 3 | The [Coordinator](../../../glossary.md#coordinator) maintains cluster state through [Raft](../../../glossary.md#raft) distributed [consensus](../../../glossary.md#consensus) and orchestrates [Director](../../../glossary.md#director) lifecycle during leadership changes. It serves as the authoritative source for cluster configuration and service directory information, enabling distributed coordination across all nodes. 4 | 5 | **Location**: [`lib/bedrock/control_plane/coordinator.ex`](../../../lib/bedrock/control_plane/coordinator.ex) 6 | 7 | ## Consensus Leadership 8 | 9 | Coordinators use Raft consensus to maintain cluster configuration and service directories. The leader Coordinator persists changes through consensus, ensuring consistent state replication across all coordinator nodes and automatic failover during leader elections. 10 | 11 | Leader readiness states prevent race conditions: `:leader_waiting_consensus` delays [Director](../../../glossary.md#director) startup until service directory population completes, then transitions to `:leader_ready` for normal operations. 12 | 13 | ## Service Directory Authority 14 | 15 | [Gateway](../../../glossary.md#gateway) nodes register services with the leader Coordinator through `register_gateway/4` or `register_services/2` operations. Service mappings replicate across all coordinators through consensus, providing Directors with complete topology information during [recovery](../../../glossary.md#recovery). 16 | 17 | ## Director Lifecycle Management 18 | 19 | Coordinator creates Directors with unique, monotonically increasing [epoch](../../../glossary.md#epoch) numbers during leadership changes. This prevents split-brain scenarios while ensuring Directors receive populated service directories before starting recovery orchestration. 20 | 21 | > **Complete Flow**: For cluster initialization and service registration sequences, see **[Cluster Startup Deep Dive](../../../deep-dives/cluster-startup.md)**. 22 | 23 | ## Related Components 24 | 25 | - **[Director](director.md)**: Recovery orchestration component created and managed by Coordinator 26 | - **[Gateway](../infrastructure/gateway.md)**: Infrastructure component that registers services with Coordinator 27 | - **[Foreman](../infrastructure/foreman.md)**: Infrastructure component coordinated through Coordinator's service directory 28 | -------------------------------------------------------------------------------- /guides/quick-reads/recovery/proxy-startup.md: -------------------------------------------------------------------------------- 1 | # Commit Proxy Startup 2 | 3 | **Deploys commit proxies for horizontal transaction scalability.** 4 | 5 | The Commit Proxy Startup phase deploys multiple [commit proxies](../../deep-dives/architecture/data-plane/commit-proxy.md) across cluster nodes to enable horizontal transaction processing. Unlike the singleton [sequencer](../../deep-dives/architecture/data-plane/sequencer.md), commit proxies can run in parallel to handle increased transaction volume. 6 | 7 | ## Scalability Through Distribution 8 | 9 | Multiple commit proxies work together to process transactions efficiently: 10 | 11 | - **Parallel Processing**: Each commit proxy handles transaction batches independently 12 | - **Load Distribution**: Incoming transactions spread across available commit proxies 13 | - **Independent Operations**: Commit proxies coordinate with [resolvers](../../deep-dives/architecture/data-plane/resolver.md) and [log servers](../../deep-dives/architecture/data-plane/log.md) separately 14 | 15 | ## Fault-Tolerant Deployment 16 | 17 | Commit proxies deploy using round-robin assignment across coordination-capable nodes to ensure: 18 | 19 | - **Physical Isolation**: Hardware failures can't eliminate all transaction processing 20 | - **Resource Distribution**: Prevents resource contention on individual nodes 21 | - **Network Resilience**: Distributed commit proxies maintain availability during network issues 22 | 23 | ## Controlled Startup 24 | 25 | Deployed commit proxies remain locked until the [Transaction System Layout](transaction-system-layout.md) phase activates them. This prevents: 26 | 27 | - **Premature Processing**: Transactions can't start before the complete pipeline is ready 28 | - **Incomplete Coordination**: Commit proxies wait for full system layout information 29 | - **Recovery Conflicts**: Locked commit proxies can't interfere with ongoing recovery 30 | 31 | ## Critical Requirements 32 | 33 | At least one commit proxy must start successfully or the cluster cannot accept transactions. The phase monitors coordination node availability and ensures minimum viable capacity before proceeding. 34 | 35 | --- 36 | 37 | **Implementation**: [`lib/bedrock/control_plane/director/recovery/proxy_startup_phase.ex`](../../../../lib/bedrock/control_plane/director/recovery/proxy_startup_phase.ex) 38 | 39 | **Next Phase**: [Resolver Startup](resolver-startup.md) - Deploy MVCC conflict detection components 40 | -------------------------------------------------------------------------------- /lib/bedrock/key_range.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.KeyRange do 2 | @moduledoc """ 3 | Utilities for working with key ranges. 4 | 5 | A key range is represented as a tuple `{start_key, end_key}` where: 6 | - `start_key` is inclusive 7 | - `end_key` is exclusive, or the atom `:end` for unbounded ranges 8 | """ 9 | 10 | alias Bedrock.Key 11 | 12 | @type t :: {Key.t(), Key.t() | :end} 13 | 14 | @doc """ 15 | Returns a range tuple {start, end} for all keys that start with the given prefix. 16 | 17 | The end key is computed using `strinc/1` to create a tight upper bound, 18 | ensuring the range includes only keys with the exact prefix. 19 | 20 | ## Examples 21 | 22 | iex> Bedrock.Key.from_prefix("user") 23 | {"user", "uses"} 24 | 25 | iex> Bedrock.Key.from_prefix("prefix/") 26 | {"prefix/", "prefix0"} 27 | 28 | ## Errors 29 | 30 | Raises an `ArgumentError` if the prefix contains only 0xFF bytes. 31 | 32 | iex> Bedrock.Key.from_prefix(<<0xFF>>) 33 | ** (ArgumentError) Key must contain at least one byte not equal to 0xFF 34 | 35 | """ 36 | @spec from_prefix(binary()) :: {binary(), binary()} 37 | def from_prefix(prefix) when is_binary(prefix), do: {prefix, Key.strinc(prefix)} 38 | 39 | @doc """ 40 | Check if two key ranges overlap. 41 | 42 | Returns true if the ranges have any keys in common. 43 | 44 | ## Examples 45 | 46 | iex> KeyRange.overlaps?({"a", "c"}, {"b", "d"}) 47 | true 48 | 49 | iex> KeyRange.overlaps?({"a", "b"}, {"c", "d"}) 50 | false 51 | 52 | iex> KeyRange.overlaps?({"a", "c"}, {"b", :end}) 53 | true 54 | """ 55 | @spec overlap?(t(), t()) :: boolean() 56 | def overlap?({_start1, end1}, {start2, :end}), do: start2 <= end1 57 | def overlap?({start1, end1}, {start2, end2}), do: start1 < end2 and start2 < end1 58 | 59 | @doc """ 60 | Check if a key range contains a specific key. 61 | 62 | Returns true if the key is within the range. 63 | 64 | ## Examples 65 | 66 | iex> KeyRange.contains?({"a", "c"}, "b") 67 | true 68 | 69 | iex> KeyRange.contains?({"a", "c"}, "c") 70 | false 71 | 72 | iex> KeyRange.contains?({"a", :end}, "z") 73 | true 74 | """ 75 | @spec contains?(t(), Key.t() | :end) :: boolean() 76 | def contains?({min_key, :end}, key), do: key >= min_key 77 | def contains?({min_key, max_key_ex}, key), do: key >= min_key and key < max_key_ex 78 | end 79 | -------------------------------------------------------------------------------- /lib/bedrock/data_plane/log/shale/state.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.Log.Shale.State do 2 | @moduledoc false 3 | 4 | alias Bedrock.ControlPlane.Director 5 | alias Bedrock.DataPlane.Log.Shale.Segment 6 | alias Bedrock.DataPlane.Log.Shale.SegmentRecycler 7 | alias Bedrock.DataPlane.Log.Shale.Writer 8 | alias Bedrock.DataPlane.Transaction 9 | alias Bedrock.Service.Foreman 10 | alias Bedrock.Service.Worker 11 | 12 | @type mode :: :locked | :running | :recovering 13 | 14 | @type t :: %__MODULE__{ 15 | cluster: module(), 16 | director: Director.ref() | nil, 17 | epoch: Bedrock.epoch() | nil, 18 | id: Worker.id(), 19 | foreman: Foreman.ref(), 20 | path: String.t(), 21 | segment_recycler: SegmentRecycler.server() | nil, 22 | # 23 | last_version: Bedrock.version(), 24 | writer: Writer.t() | nil, 25 | active_segment: Segment.t() | nil, 26 | segments: [Segment.t()], 27 | pending_pushes: %{ 28 | Bedrock.version() => 29 | {encoded_transaction :: Transaction.encoded(), ack_fn :: (:ok | {:error, term()} -> :ok)} 30 | }, 31 | # 32 | mode: mode(), 33 | oldest_version: Bedrock.version(), 34 | otp_name: Worker.otp_name(), 35 | params: %{ 36 | default_pull_limit: pos_integer(), 37 | max_pull_limit: pos_integer() 38 | }, 39 | waiting_pullers: %{ 40 | Bedrock.version() => [ 41 | {Bedrock.timestamp_in_ms(), reply_to_fn :: (any() -> :ok), opts :: [limit: integer(), timeout: timeout()]} 42 | ] 43 | } 44 | } 45 | defstruct cluster: nil, 46 | director: nil, 47 | epoch: nil, 48 | foreman: nil, 49 | id: nil, 50 | path: nil, 51 | segment_recycler: nil, 52 | # 53 | last_version: nil, 54 | writer: nil, 55 | segments: [], 56 | active_segment: nil, 57 | pending_pushes: %{}, 58 | # 59 | mode: :locked, 60 | oldest_version: nil, 61 | otp_name: nil, 62 | pending_transactions: %{}, 63 | waiting_pullers: %{}, 64 | params: %{ 65 | default_pull_limit: 100, 66 | max_pull_limit: 500 67 | } 68 | end 69 | -------------------------------------------------------------------------------- /.github/workflows/elixir_ci.yaml: -------------------------------------------------------------------------------- 1 | name: Elixir CI 2 | 3 | on: 4 | push: 5 | branches: [ "main", "develop" ] 6 | pull_request: 7 | branches: [ "main", "develop" ] 8 | 9 | env: 10 | MIX_ENV: test 11 | GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} 12 | 13 | permissions: 14 | contents: read 15 | 16 | jobs: 17 | test: 18 | runs-on: ubuntu-24.04 19 | name: Test on OTP ${{matrix.otp}} / Elixir ${{matrix.elixir}} 20 | strategy: 21 | matrix: 22 | include: 23 | - elixir: "1.18.4" 24 | otp: "27.2" 25 | steps: 26 | 27 | - name: Checkout code 28 | uses: actions/checkout@v4 29 | 30 | - name: Install Elixir and Erlang 31 | uses: erlef/setup-beam@v1 32 | with: 33 | elixir-version: ${{ matrix.elixir }} 34 | otp-version: ${{ matrix.otp }} 35 | install-hex: true 36 | install-rebar: true 37 | 38 | - name: Restore deps and _build cache 39 | uses: actions/cache@v3 40 | with: 41 | path: | 42 | deps 43 | _build 44 | key: ${{ runner.os }}-${{ matrix.elixir }}-${{ matrix.otp }}-${{ hashFiles('**/mix.lock') }} 45 | restore-keys: | 46 | ${{ runner.os }}-${{ matrix.elixir }}-${{ matrix.otp }}- 47 | 48 | - name: Clean to rule out incremental build as a source of flakiness 49 | if: github.run_attempt != '1' 50 | run: | 51 | mix deps.clean --all 52 | mix clean 53 | shell: sh 54 | 55 | - name: Install dependencies 56 | run: mix deps.get 57 | 58 | - name: Compiles without warnings 59 | run: mix compile --warnings-as-errors 60 | 61 | - name: Check Formatting 62 | run: mix format --check-formatted 63 | 64 | - name: Credo 65 | run: mix credo --strict 66 | 67 | # Define how to cache the `priv/plts` directory. After the first run, 68 | # this speeds up dialyzer runs a lot. 69 | - name: Cache PLTs 70 | id: cache-plts 71 | uses: actions/cache@v3 72 | env: 73 | cache-name: cache-plts 74 | with: 75 | path: priv/plts 76 | key: ${{ runner.os }}-mix-${{ env.cache-name }}-${{ hashFiles('**/mix.lock') }} 77 | restore-keys: | 78 | ${{ runner.os }}-mix-${{ env.cache-name }}- 79 | ${{ runner.os }}-mix- 80 | 81 | - name: Dialyzer 82 | run: mix dialyzer --halt-exit-status 83 | 84 | - name: Run tests 85 | run: mix coveralls.github -------------------------------------------------------------------------------- /lib/bedrock/service/foreman/working_directory.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.Service.Foreman.WorkingDirectory do 2 | @moduledoc false 3 | import Bedrock.Service.Manifest, only: [load_from_file: 1] 4 | 5 | alias Bedrock.Service.Manifest 6 | alias Bedrock.Service.Worker 7 | 8 | @spec initialize_working_directory(Path.t(), Manifest.t()) :: 9 | :ok | {:error, File.posix()} 10 | def initialize_working_directory(working_directory, manifest) do 11 | path_to_manifest = Path.join(working_directory, "manifest.json") 12 | 13 | with :ok <- File.mkdir_p(working_directory), 14 | :ok <- manifest.worker.one_time_initialization(working_directory) do 15 | Manifest.write_to_file(manifest, path_to_manifest) 16 | end 17 | end 18 | 19 | @spec read_and_validate_manifest(Path.t(), Worker.id(), cluster_name :: String.t()) :: 20 | {:ok, Manifest.t()} 21 | | {:error, 22 | :cluster_name_in_manifest_does_not_match 23 | | :id_in_manifest_does_not_match 24 | | :invalid_cluster_id 25 | | :invalid_cluster_name 26 | | :invalid_worker_name 27 | | :manifest_does_not_exist 28 | | :manifest_is_invalid 29 | | :manifest_is_not_a_dictionary 30 | | :worker_module_does_not_exist 31 | | :worker_module_does_not_implement_behaviour 32 | | :worker_module_failed_to_load 33 | | :worker_module_is_invalid} 34 | def read_and_validate_manifest(path, worker_id, cluster_name) do 35 | with {:ok, manifest} <- load_from_file(Path.join(path, "manifest.json")), 36 | :ok <- check_manifest_id(manifest, worker_id), 37 | :ok <- check_manifest_cluster_name(manifest, cluster_name) do 38 | {:ok, manifest} 39 | end 40 | end 41 | 42 | @spec check_manifest_id(manifest :: Manifest.t(), id :: Worker.id()) :: 43 | :ok | {:error, :id_in_manifest_does_not_match} 44 | defp check_manifest_id(%{id: id}, id), do: :ok 45 | defp check_manifest_id(_, _), do: {:error, :id_in_manifest_does_not_match} 46 | 47 | @spec check_manifest_cluster_name(manifest :: Manifest.t(), cluster_name :: String.t()) :: 48 | :ok | {:error, :cluster_name_in_manifest_does_not_match} 49 | defp check_manifest_cluster_name(%{cluster: cluster_name}, cluster_name), do: :ok 50 | defp check_manifest_cluster_name(_, _), do: {:error, :cluster_name_in_manifest_does_not_match} 51 | end 52 | -------------------------------------------------------------------------------- /test/bedrock/internal/atomics_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.Internal.AtomicsTest do 2 | use ExUnit.Case, async: true 3 | 4 | alias Bedrock.Internal.Atomics 5 | 6 | describe "add/2" do 7 | test "adds little-endian binary values" do 8 | assert Atomics.add(<<5>>, <<3>>) == <<8>> 9 | end 10 | 11 | test "adds with carry propagation" do 12 | assert Atomics.add(<<255>>, <<1>>) == <<0, 1>> 13 | end 14 | 15 | test "handles missing existing value" do 16 | assert Atomics.add(<<>>, <<5, 0>>) == <<5, 0>> 17 | end 18 | 19 | test "handles empty operand" do 20 | assert Atomics.add(<<5, 0>>, <<>>) == <<>> 21 | end 22 | 23 | test "pads existing to match operand length" do 24 | assert Atomics.add(<<5>>, <<3, 1>>) == <<8, 1>> 25 | end 26 | end 27 | 28 | describe "min/2" do 29 | test "returns minimum of two values" do 30 | assert Atomics.min(<<5>>, <<3>>) == <<3>> 31 | end 32 | 33 | test "returns operand when existing is missing" do 34 | assert Atomics.min(<<>>, <<10, 0>>) == <<10, 0>> 35 | end 36 | 37 | test "handles empty operand" do 38 | assert Atomics.min(<<5>>, <<>>) == <<>> 39 | end 40 | 41 | test "pads existing to operand length for comparison" do 42 | assert Atomics.min(<<5>>, <<0, 1>>) == <<5, 0>> 43 | end 44 | 45 | test "compares multi-byte values correctly" do 46 | # 256 (<<0, 1>>) vs 255 (<<255>>) - 255 padded should be minimum 47 | assert Atomics.min(<<255>>, <<0, 1>>) == <<255, 0>> 48 | end 49 | end 50 | 51 | describe "max/2" do 52 | test "returns maximum of two values" do 53 | assert Atomics.max(<<5>>, <<3>>) == <<5>> 54 | end 55 | 56 | test "returns operand when existing is missing" do 57 | assert Atomics.max(<<>>, <<10, 0>>) == <<10, 0>> 58 | end 59 | 60 | test "handles empty operand" do 61 | assert Atomics.max(<<5>>, <<>>) == <<>> 62 | end 63 | 64 | test "pads existing to operand length for comparison" do 65 | assert Atomics.max(<<5>>, <<0, 1>>) == <<0, 1>> 66 | end 67 | 68 | test "compares multi-byte values correctly" do 69 | # 256 (<<0, 1>>) vs 255 (<<255>>) - 256 should be maximum 70 | assert Atomics.max(<<255>>, <<0, 1>>) == <<0, 1>> 71 | # 65537 (<<1, 0, 1>>) vs 65536 (<<0, 0, 1>>) - 65537 should be maximum 72 | assert Atomics.max(<<0, 0, 1>>, <<1, 0, 1>>) == <<1, 0, 1>> 73 | end 74 | end 75 | end 76 | -------------------------------------------------------------------------------- /lib/bedrock/data_plane/commit_proxy.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.CommitProxy do 2 | @moduledoc """ 3 | Central coordinator of Bedrock's transaction commit process. 4 | 5 | The Commit Proxy batches transactions from multiple clients for efficient processing, 6 | orchestrates conflict resolution through Resolvers, and ensures durable persistence 7 | across all required log servers. It transforms individual transaction requests into 8 | efficiently processed batches while maintaining strict consistency guarantees. 9 | 10 | Transaction batching creates a fundamental trade-off between latency and throughput. 11 | The Commit Proxy manages this through configurable size and time limits that balance 12 | responsiveness against processing efficiency. This batching strategy enables 13 | intra-batch conflict detection and amortizes the fixed costs of conflict resolution 14 | and logging across multiple transactions while preserving the arrival order of 15 | transactions within each batch. 16 | 17 | The component uses a fail-fast recovery model where unrecoverable errors trigger 18 | process exit and Director-coordinated recovery. Commit Proxies start in locked mode 19 | and require explicit unlocking through `recover_from/3` before accepting transaction 20 | commits, ensuring proper coordination during cluster recovery scenarios. 21 | 22 | For detailed architectural concepts and design reasoning, see the 23 | [Commit Proxy documentation](../../../../docs/components/commit-proxy.md). 24 | """ 25 | 26 | use Bedrock.Internal.GenServerApi, for: __MODULE__.Server 27 | 28 | alias Bedrock.ControlPlane.Config.TransactionSystemLayout 29 | alias Bedrock.DataPlane.Transaction 30 | 31 | @type ref :: pid() | atom() | {atom(), node()} 32 | 33 | @spec recover_from( 34 | commit_proxy_ref :: ref(), 35 | lock_token :: binary(), 36 | transaction_system_layout :: TransactionSystemLayout.t() 37 | ) :: :ok | {:error, :timeout} | {:error, :unavailable} 38 | def recover_from(commit_proxy, lock_token, transaction_system_layout), 39 | do: call(commit_proxy, {:recover_from, lock_token, transaction_system_layout}, :infinity) 40 | 41 | @spec commit(commit_proxy_ref :: ref(), transaction :: Transaction.encoded()) :: 42 | {:ok, version :: Bedrock.version(), index :: non_neg_integer()} | {:error, :timeout | :unavailable} 43 | def commit(commit_proxy, transaction), do: call(commit_proxy, {:commit, transaction}, :infinity) 44 | end 45 | -------------------------------------------------------------------------------- /lib/bedrock/data_plane/storage/basalt/keyspace.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.Storage.Basalt.Keyspace do 2 | @moduledoc """ 3 | A keyspace is an ordered set of keys. 4 | 5 | Under the hood, an ordered_set ETS table is used to store the keys. It is 6 | intended that the keyspace can be read and written concurrently by multiple 7 | processes, so we rely on the fact that ETS insert operations are atomic to 8 | ensure that the keyspace is always in a consistent state. As such, each key 9 | is stored along with a boolean "presence" value. This allows us to determine 10 | whether a key has been deleted or not (while still relying on the atomicity 11 | of the insert operation). 12 | """ 13 | 14 | alias Bedrock.DataPlane.Transaction 15 | 16 | @opaque t :: :ets.tid() 17 | 18 | @spec new(atom()) :: t() 19 | def new(name) when is_atom(name), do: :ets.new(name, [:ordered_set, :public, read_concurrency: true]) 20 | 21 | @spec close(pkv :: t()) :: :ok 22 | def close(mvcc) do 23 | :ets.delete(mvcc) 24 | :ok 25 | end 26 | 27 | @spec apply_transaction(keyspace :: t(), Transaction.encoded()) :: :ok 28 | def apply_transaction(keyspace, encoded_transaction) do 29 | {:ok, version} = Transaction.commit_version(encoded_transaction) 30 | {:ok, mutations_stream} = Transaction.mutations(encoded_transaction) 31 | 32 | # Convert mutations to key presence indicators 33 | key_entries = 34 | Enum.map(mutations_stream, fn 35 | {:set, key, _value} -> {key, true} 36 | # Treat as single key clear for simplicity 37 | {:clear_range, key, _end} -> {key, false} 38 | end) 39 | 40 | true = :ets.insert(keyspace, [{:last_version, version} | key_entries]) 41 | :ok 42 | end 43 | 44 | @spec insert_many(keyspace :: t(), keys :: [Bedrock.key()]) :: :ok 45 | def insert_many(keyspace, keys) do 46 | true = :ets.insert_new(keyspace, Enum.map(keys, fn key -> {key, true} end)) 47 | :ok 48 | end 49 | 50 | @spec prune(keyspace :: t()) :: {:ok, n_pruned :: non_neg_integer()} 51 | def prune(keyspace) do 52 | n_pruned = :ets.select_delete(keyspace, [{{:_, :"$1"}, [{:"=:=", false, :"$1"}], [true]}]) 53 | {:ok, n_pruned} 54 | end 55 | 56 | @spec key_exists?(keyspace :: t(), Bedrock.key()) :: boolean() 57 | def key_exists?(keyspace, key) when is_binary(key) do 58 | keyspace 59 | |> :ets.lookup(key) 60 | |> case do 61 | [] -> false 62 | [{_, present}] -> present 63 | end 64 | end 65 | end 66 | -------------------------------------------------------------------------------- /test/bedrock/data_plane/commit_proxy/sequencer_notification_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.CommitProxy.SequencerNotificationTest do 2 | use ExUnit.Case, async: true 3 | 4 | alias Bedrock.DataPlane.CommitProxy.Batch 5 | alias Bedrock.DataPlane.CommitProxy.Finalization 6 | alias Bedrock.DataPlane.CommitProxy.LayoutOptimization 7 | 8 | # Common test setup 9 | defp create_batch do 10 | %Batch{ 11 | commit_version: 100, 12 | last_commit_version: 99, 13 | n_transactions: 0, 14 | buffer: [] 15 | } 16 | end 17 | 18 | defp create_transaction_system_layout(sequencer) do 19 | %{ 20 | sequencer: sequencer, 21 | resolvers: [], 22 | logs: %{}, 23 | storage_teams: [], 24 | services: %{} 25 | } 26 | end 27 | 28 | defp create_finalization_opts do 29 | [ 30 | epoch: 1, 31 | precomputed: LayoutOptimization.precompute_from_layout(%{resolvers: []}), 32 | resolver_fn: fn _, _, _, _, _, _ -> {:ok, []} end, 33 | batch_log_push_fn: fn _, _, _, _, _ -> :ok end 34 | ] 35 | end 36 | 37 | defp create_mock_sequencer do 38 | test_pid = self() 39 | 40 | spawn(fn -> 41 | receive do 42 | {:"$gen_call", from, {:report_successful_commit, version}} -> 43 | GenServer.reply(from, :ok) 44 | send(test_pid, {:sequencer_notified, version}) 45 | after 46 | 1000 -> :timeout 47 | end 48 | end) 49 | end 50 | 51 | describe "finalize_batch/3" do 52 | test "notifies sequencer after log persistence" do 53 | mock_sequencer = create_mock_sequencer() 54 | batch = create_batch() 55 | layout = create_transaction_system_layout(mock_sequencer) 56 | opts = create_finalization_opts() 57 | 58 | assert {:ok, 0, 0} = Finalization.finalize_batch(batch, layout, opts) 59 | assert_receive {:sequencer_notified, 100}, 100 60 | 61 | Process.exit(mock_sequencer, :kill) 62 | end 63 | 64 | test "returns error when sequencer ref is invalid" do 65 | # Documents current behavior: GenServer.call returns {:error, :unavailable} 66 | # for invalid refs, so sequencer notification returns an error but doesn't crash the commit proxy. 67 | batch = create_batch() 68 | layout = create_transaction_system_layout(:invalid_sequencer_ref) 69 | opts = create_finalization_opts() 70 | 71 | assert {:error, :unavailable} = Finalization.finalize_batch(batch, layout, opts) 72 | end 73 | end 74 | end 75 | -------------------------------------------------------------------------------- /lib/bedrock/type_coercion.ex: -------------------------------------------------------------------------------- 1 | defprotocol Bedrock.ToKeyRange do 2 | @moduledoc """ 3 | Protocol for converting various types to key ranges. 4 | 5 | A key range is represented as a tuple `{start_key, end_key}` where: 6 | - `start_key` is inclusive 7 | - `end_key` is exclusive, or the atom `:end` for unbounded ranges 8 | 9 | This protocol provides a unified interface for range operations in Bedrock, 10 | allowing directories, keyspaces, and other types to be used wherever a 11 | key range is expected. 12 | """ 13 | 14 | @type key_range :: {binary(), binary() | :end} 15 | 16 | @doc """ 17 | Converts the given data to a key range tuple. 18 | 19 | Returns a tuple `{start_key, end_key}` where `end_key` may be a binary 20 | or the atom `:end` for unbounded ranges. 21 | """ 22 | @spec to_key_range(term()) :: key_range() 23 | def to_key_range(data) 24 | end 25 | 26 | defprotocol Bedrock.ToKeyspace do 27 | @moduledoc """ 28 | Protocol for converting various types to keyspaces. 29 | 30 | This protocol allows directories, prefixes, and other types to be 31 | automatically coerced into `Bedrock.Keyspace` structs, enabling 32 | more ergonomic APIs. 33 | """ 34 | 35 | @doc """ 36 | Converts the given data to a keyspace. 37 | 38 | Returns a `Bedrock.Keyspace` struct that can be used for data operations. 39 | """ 40 | @spec to_keyspace(term()) :: Bedrock.Keyspace.t() 41 | def to_keyspace(data) 42 | end 43 | 44 | # Implementation for tuples (explicit ranges) 45 | defimpl Bedrock.ToKeyRange, for: Tuple do 46 | def to_key_range({start_key, end_key}) when is_binary(start_key) and is_binary(end_key) and start_key < end_key, 47 | do: {start_key, end_key} 48 | 49 | def to_key_range(tuple) do 50 | raise ArgumentError, "ToKeyRange expects a 2-tuple with binary start key, got: #{inspect(tuple)}" 51 | end 52 | end 53 | 54 | # Implementation for binaries (prefix ranges) 55 | defimpl Bedrock.ToKeyRange, for: BitString do 56 | alias Bedrock.KeyRange 57 | 58 | def to_key_range(prefix) when is_binary(prefix), do: KeyRange.from_prefix(prefix) 59 | end 60 | 61 | # ToKeyspace implementations 62 | 63 | # Implementation for Keyspace (identity) 64 | defimpl Bedrock.ToKeyspace, for: Bedrock.Keyspace do 65 | def to_keyspace(keyspace), do: keyspace 66 | end 67 | 68 | # Implementation for binaries (create keyspace from prefix) 69 | defimpl Bedrock.ToKeyspace, for: BitString do 70 | def to_keyspace(prefix) when is_binary(prefix), do: Bedrock.Keyspace.new(prefix) 71 | end 72 | -------------------------------------------------------------------------------- /test/bedrock/data_plane/log/shale/pull_boundary_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.Log.Shale.PullBoundaryTest do 2 | use ExUnit.Case, async: false 3 | 4 | alias Bedrock.DataPlane.Log 5 | alias Bedrock.DataPlane.Log.Shale.Server 6 | alias Bedrock.DataPlane.Version 7 | 8 | @moduletag :tmp_dir 9 | 10 | setup %{tmp_dir: tmp_dir} do 11 | cluster = Bedrock.Cluster 12 | otp_name = :"boundary_test_#{System.unique_integer([:positive])}" 13 | id = "boundary_log_#{System.unique_integer([:positive])}" 14 | foreman = self() 15 | path = Path.join(tmp_dir, "log_segments") 16 | 17 | File.mkdir_p!(path) 18 | 19 | # Start the Shale server 20 | {:ok, pid} = 21 | GenServer.start_link( 22 | Server, 23 | {cluster, otp_name, id, foreman, path, true}, 24 | name: otp_name 25 | ) 26 | 27 | # Wait for initialization to complete 28 | Process.sleep(10) 29 | 30 | on_exit(fn -> 31 | if Process.alive?(pid) do 32 | GenServer.stop(pid) 33 | end 34 | end) 35 | 36 | {:ok, log: pid, path: path} 37 | end 38 | 39 | describe "empty log pull boundary conditions" do 40 | test "pull from version 0 returns error immediately without timeout", %{log: log} do 41 | # Pull from version 0 on empty log should return error immediately 42 | # (version 0 == last_version, no transactions after it, no willing_to_wait) 43 | assert {:error, :version_too_new} = Log.pull(log, Version.from_integer(0), []) 44 | end 45 | 46 | test "pull from version 0 with timeout returns error immediately without waiting", %{log: log} do 47 | start_time = System.monotonic_time(:millisecond) 48 | assert {:error, :version_too_new} = Log.pull(log, Version.from_integer(0), timeout_in_ms: 100) 49 | elapsed = System.monotonic_time(:millisecond) - start_time 50 | 51 | assert elapsed < 50, "Should return immediately, but took #{elapsed}ms" 52 | end 53 | 54 | test "pull from higher version returns version_too_new", %{log: log} do 55 | assert {:error, :version_too_new} = Log.pull(log, Version.from_integer(1), []) 56 | end 57 | end 58 | 59 | describe "log info/2" do 60 | test "returns expected structure on empty log", %{log: log} do 61 | # Verify the log can provide info without crashing and returns expected structure 62 | assert {:ok, %{last_version: <<0, 0, 0, 0, 0, 0, 0, 0>>, oldest_version: _}} = 63 | Log.info(log, [:last_version, :oldest_version]) 64 | end 65 | end 66 | end 67 | -------------------------------------------------------------------------------- /guides/quick-reads/recovery/storage-recruitment.md: -------------------------------------------------------------------------------- 1 | # Storage Recruitment 2 | 3 | **Assigns storage services to vacancies while preserving existing data.** 4 | 5 | Storage recruitment fills [vacancy placeholders](vacancy-creation.md) with concrete storage services, following strict data preservation rules that protect committed state from the previous system. 6 | 7 | ## Core Principle: Data Safety First 8 | 9 | Unlike [log recruitment](log-recruitment.md), storage recruitment must handle services containing irreplaceable committed data—user records, computed indexes, and historical state that cannot be recreated. The recruitment algorithm prioritizes data preservation over efficiency. 10 | 11 | ## Three-Tier Assignment Strategy 12 | 13 | **Absolute Preservation**: All storage services from the previous [transaction system layout](transaction-system-layout.md) are excluded from recruitment. These contain valuable data and remain untouched for future access. 14 | 15 | **Conservative Reuse**: Available storage services not part of the previous system become candidates for filling vacancies. Their state is treated with caution. 16 | 17 | **Controlled Creation**: When existing services cannot satisfy requirements, new storage workers are created using fault-tolerant round-robin placement across available nodes. 18 | 19 | ## Recruitment Process 20 | 21 | 1. **Vacancy Assessment**: Catalog all storage vacancy placeholders requiring concrete service assignments 22 | 2. **Candidate Pool Analysis**: Identify available services, excluding previous-system services and already-assigned services 23 | 3. **Assignment Strategy**: Fill vacancies with candidates, creating new workers if needed 24 | 4. **Service Locking**: All recruited services acquire [service locks](service-locking.md) for exclusive control 25 | 26 | ## Error Handling 27 | 28 | **Resource Insufficiency**: Recruitment stalls if insufficient nodes exist for fault-tolerant worker creation. 29 | 30 | **Locking Failures**: If any service fails to acquire its lock, the entire phase stalls to prevent partial control scenarios. 31 | 32 | **Epoch Supersession**: If a recruited service reports a newer epoch, recovery terminates immediately with `:newer_epoch_exists`. 33 | 34 | ## Next Phase 35 | 36 | Upon completion, all storage vacancies are filled with locked services. Recovery proceeds to [log replay](log-replay.md) to copy committed transaction data from the previous system. 37 | 38 | --- 39 | 40 | **Implementation**: `lib/bedrock/control_plane/director/recovery/storage_recruitment_phase.ex` 41 | -------------------------------------------------------------------------------- /lib/bedrock/data_plane/resolver.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.Resolver do 2 | @moduledoc """ 3 | MVCC conflict detection engine for Bedrock's optimistic concurrency control system. 4 | 5 | The Resolver detects read-write and write-write conflicts by maintaining an interval 6 | tree that tracks which key ranges were written at which versions. It processes 7 | transaction batches from Commit Proxies and returns lists of conflicting transaction 8 | indices to abort. 9 | 10 | Resolvers start in running mode and are immediately ready to process transactions. 11 | They handle out-of-order transactions through a version-indexed waiting queue that 12 | ensures consistent conflict detection regardless of network timing variations. 13 | 14 | For detailed conflict detection concepts and architectural integration, see the 15 | [Resolver documentation](../../../../docs/components/resolver.md). 16 | """ 17 | 18 | alias Bedrock.DataPlane.Transaction 19 | 20 | @type ref :: pid() | atom() | {atom(), node()} 21 | 22 | @spec resolve_transactions( 23 | ref(), 24 | epoch :: Bedrock.epoch(), 25 | last_version :: Bedrock.version(), 26 | commit_version :: Bedrock.version(), 27 | [Transaction.encoded()], 28 | opts :: [timeout: Bedrock.timeout_in_ms()] 29 | ) :: 30 | {:ok, aborted :: [transaction_index :: non_neg_integer()]} 31 | | {:failure, :timeout, ref()} 32 | | {:failure, :unavailable, ref()} 33 | def resolve_transactions(ref, epoch, last_version, commit_version, transaction_summaries, opts \\ []) do 34 | timeout = opts[:timeout] || :infinity 35 | 36 | :telemetry.span( 37 | [:bedrock, :data_plane, :resolver, :call, :resolve_transactions], 38 | %{ 39 | resolver_id: ref, 40 | epoch: epoch, 41 | last_version: last_version, 42 | commit_version: commit_version, 43 | transaction_summaries: transaction_summaries, 44 | timeout_ms: timeout 45 | }, 46 | fn -> 47 | ref 48 | |> GenServer.call( 49 | {:resolve_transactions, epoch, {last_version, commit_version}, transaction_summaries}, 50 | timeout 51 | ) 52 | |> case do 53 | {:ok, aborted} -> {{:ok, aborted}, %{aborted: aborted}} 54 | {:error, reason} -> {{:error, reason}, %{}} 55 | end 56 | end 57 | ) 58 | catch 59 | :exit, {:timeout, _} -> {:failure, :timeout, ref} 60 | :exit, _reason -> {:failure, :unavailable, ref} 61 | end 62 | end 63 | -------------------------------------------------------------------------------- /guides/README.md: -------------------------------------------------------------------------------- 1 | # Welcome to Bedrock 2 | 3 | Bedrock is an embedded, distributed key-value store that goes beyond traditional ACID guarantees. It provides consistent reads, strict serialization, and transactions across the entire key-space with a simple API. 4 | 5 | This documentation is organized to help you find exactly what you need, when you need it. Whether you're getting started, implementing features, or diving deep into the architecture, we've structured the information to match how you actually work. 6 | 7 | ## How This Documentation Works 8 | 9 | We organize information into three levels: 10 | 11 | **Quick Reads** - Short, focused explanations that get you oriented fast. Perfect for understanding concepts or explaining Bedrock to others. 12 | 13 | **Guides** - Step-by-step instructions and practical information for building with Bedrock. These contain everything you need to implement features or integrate components. 14 | 15 | **Deep Dives** - Comprehensive technical coverage with full architectural details. Use these when you need to understand the "why" behind the design or troubleshoot complex issues. 16 | 17 | ## Getting Started 18 | 19 | Start here to understand what Bedrock is and how it works: 20 | 21 | - **[User's Perspective](quick-reads/users-perspective.md)** - How you'll actually use Bedrock in your applications 22 | - **[Transaction Basics](quick-reads/transactions.md)** - Core concepts of MVCC and how transactions work 23 | 24 | ## Understanding Bedrock 25 | 26 | For comprehensive technical coverage: 27 | 28 | - **[Architecture Analysis](deep-dives/architecture.md)** - Complete architectural patterns and design principles 29 | - **[Control Plane Overview](quick-reads/control-plane.md)** - How cluster coordination and consensus work 30 | - **[Data Plane Overview](quick-reads/data-plane.md)** - Transaction processing and data persistence 31 | - **[Storage Implementations](deep-dives/architecture/implementations/README.md)** - Available storage engines and their 32 | - **[Transaction System Layout](quick-reads/transaction-system-layout.md)** - The big picture of how components work together 33 | - **[Transaction Processing](deep-dives/transactions.md)** - Full details of MVCC implementation and commit protocols 34 | - **[Cluster Startup](deep-dives/cluster-startup.md)** - Bootstrap processes and system initialization 35 | - **[Recovery](quick-reads/recovery.md)** - High-level understanding of system resilience ([More Detail](deep-dives/recovery.md)) 36 | 37 | ## Reference 38 | 39 | - **[Glossary](glossary.md)** - Complete terminology reference with cross-linked definitions 40 | -------------------------------------------------------------------------------- /guides/quick-reads/recovery/version-determination.md: -------------------------------------------------------------------------------- 1 | # Version Determination 2 | 3 | **Finding the highest transaction version guaranteed durable across the cluster.** 4 | 5 | Version determination solves a critical distributed systems challenge: identifying the maximum [version](../../glossary.md#version) safely recoverable when [storage teams](../../glossary.md#storage-team) report different durability states. This establishes the recovery baseline—the foundation for all subsequent phases. 6 | 7 | ## The Challenge 8 | 9 | Storage servers inevitably operate at different versions due to network delays and processing variations. Recovery cannot simply use the highest reported version, as that replica might fail during reconstruction. Instead, it must find a fault-tolerant baseline where sufficient replicas guarantee data availability. 10 | 11 | ## Mathematical Approach 12 | 13 | The algorithm uses two-tier analysis: 14 | 15 | ### Storage Team Analysis 16 | 17 | For each team, recovery applies quorum mathematics to replica versions. In a three-replica team reporting versions [95, 98, 100], the durable version becomes 98—ensuring two replicas have the data even if the highest-version replica fails. 18 | 19 | ### Cluster-Wide Baseline 20 | 21 | The cluster baseline takes the minimum across all team baselines, ensuring every data piece remains available throughout the cluster. 22 | 23 | ## Team Health Classification 24 | 25 | Teams are classified during analysis: 26 | 27 | - **Healthy**: Exact replica count, optimal operation 28 | - **Degraded**: Over/under-replicated, needs rebalancing 29 | - **Insufficient**: Below quorum threshold, causes recovery termination 30 | 31 | ## Guarantees 32 | 33 | The conservative approach prioritizes availability over maximizing recoverable versions. The established baseline guarantees data access even with additional failures during reconstruction. 34 | 35 | ## Integration 36 | 37 | Version determination outputs enable subsequent phases: 38 | 39 | - Baseline informs [log replay](log-replay.md) boundaries 40 | - Team health guides [storage recruitment](storage-recruitment.md) planning 41 | - Durable version sets [sequencer startup](sequencer-startup.md) baseline 42 | 43 | ## Implementation 44 | 45 | **Input**: Storage team layouts, recovery info, replication configuration 46 | **Output**: Durable version baseline, degraded team list 47 | **Source**: `lib/bedrock/control_plane/director/recovery/version_determination_phase.ex` 48 | 49 | --- 50 | 51 | **Next**: [Log Recruitment](log-recruitment.md) - Replacing vacancy placeholders with operational log services 52 | -------------------------------------------------------------------------------- /guides/quick-reads/recovery/resolver-startup.md: -------------------------------------------------------------------------------- 1 | # Resolver Startup 2 | 3 | **Starting MVCC conflict detection components to prevent transaction conflicts.** 4 | 5 | The resolver startup phase converts abstract [resolver](../../deep-dives/architecture/data-plane/resolver.md) descriptors from [vacancy creation](vacancy-creation.md) into operational processes that implement [Multi-Version Concurrency Control (MVCC)](../../glossary.md#multi-version-concurrency-control) conflict detection across the distributed system. 6 | 7 | ## What Happens 8 | 9 | Recovery transforms resolver placeholders into working components through three key operations: 10 | 11 | **Keyspace Assignment**: Resolvers receive specific [key ranges](../../glossary.md#key-range) with complete coverage—every possible key has exactly one responsible resolver for conflict detection. 12 | 13 | **Log Assignment**: Each resolver gets assigned the minimal set of transaction logs containing data relevant to their key ranges, using tag-based filtering to avoid unnecessary data transfer. 14 | 15 | **Historical State Recovery**: Resolvers rebuild their complete MVCC state by processing historical transactions within the recovery [version](../../glossary.md#version) range, enabling them to detect conflicts between pre-recovery and post-recovery transactions. 16 | 17 | ## Process Distribution 18 | 19 | Recovery distributes resolver processes across resolution-capable nodes using round-robin assignment for fault tolerance. Each resolver receives a lock token and immediately begins recovering its conflict detection state from assigned transaction logs. 20 | 21 | ## Critical Requirements 22 | 23 | Resolvers must rebuild complete historical state to maintain MVCC consistency across the recovery boundary. Without this, the system could not detect conflicts between old and new transactions, potentially allowing data corruption. 24 | 25 | Recovery fails fast if resolution-capable nodes become unavailable or individual resolver startup fails—transaction isolation guarantees cannot be maintained without complete conflict detection coverage. 26 | 27 | ## Implementation 28 | 29 | **Source**: `lib/bedrock/control_plane/director/recovery/resolver_startup_phase.ex` 30 | 31 | **Key Inputs**: Resolver descriptors, storage team assignments, transaction logs, version vector bounds 32 | 33 | **Output**: Operational resolver processes ready for MVCC conflict detection 34 | 35 | ## Next Phase 36 | 37 | Recovery proceeds to [Transaction System Layout](transaction-system-layout.md), where operational resolvers join the coordination blueprint for distributed transaction processing. 38 | -------------------------------------------------------------------------------- /lib/bedrock/data_plane/log/telemetry.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.Log.Telemetry do 2 | @moduledoc false 3 | alias Bedrock.DataPlane.Log 4 | alias Bedrock.DataPlane.Transaction 5 | alias Bedrock.Telemetry 6 | 7 | @spec trace_metadata() :: map() 8 | def trace_metadata, do: Process.get(:trace_metadata, %{}) 9 | 10 | @spec trace_metadata(metadata :: map()) :: map() 11 | def trace_metadata(metadata), do: Process.put(:trace_metadata, Enum.into(metadata, trace_metadata())) 12 | 13 | @spec trace_started() :: :ok 14 | def trace_started, do: Telemetry.execute([:bedrock, :log, :started], %{}, trace_metadata()) 15 | 16 | @spec trace_lock_for_recovery(epoch :: Bedrock.epoch()) :: :ok 17 | def trace_lock_for_recovery(epoch) do 18 | Telemetry.execute( 19 | [:bedrock, :log, :lock_for_recovery], 20 | %{}, 21 | Map.put(trace_metadata(), :epoch, epoch) 22 | ) 23 | end 24 | 25 | @spec trace_recover_from( 26 | source_log :: Log.ref(), 27 | first_version :: Bedrock.version(), 28 | last_version :: Bedrock.version() 29 | ) :: :ok 30 | def trace_recover_from(source_log, first_version, last_version) do 31 | Telemetry.execute( 32 | [:bedrock, :log, :recover_from], 33 | %{}, 34 | Map.merge(trace_metadata(), %{ 35 | source_log: source_log, 36 | first_version: first_version, 37 | last_version: last_version 38 | }) 39 | ) 40 | end 41 | 42 | @spec trace_push_transaction(transaction :: Transaction.encoded()) :: :ok 43 | def trace_push_transaction(transaction) when is_binary(transaction) do 44 | Telemetry.execute( 45 | [:bedrock, :log, :push], 46 | %{}, 47 | Map.put(trace_metadata(), :transaction, transaction) 48 | ) 49 | end 50 | 51 | @spec trace_push_out_of_order( 52 | expected_version :: Bedrock.version(), 53 | current_version :: Bedrock.version() 54 | ) :: :ok 55 | def trace_push_out_of_order(expected_version, current_version) do 56 | Telemetry.execute( 57 | [:bedrock, :log, :push_out_of_order], 58 | %{}, 59 | Map.merge(trace_metadata(), %{ 60 | expected_version: expected_version, 61 | current_version: current_version 62 | }) 63 | ) 64 | end 65 | 66 | @spec trace_pull_transactions(from_version :: Bedrock.version(), opts :: Keyword.t()) :: :ok 67 | def trace_pull_transactions(from_version, opts) do 68 | Telemetry.execute( 69 | [:bedrock, :log, :pull], 70 | %{}, 71 | Map.merge(trace_metadata(), %{ 72 | from_version: from_version, 73 | opts: opts 74 | }) 75 | ) 76 | end 77 | end 78 | -------------------------------------------------------------------------------- /lib/bedrock/data_plane/log/shale/long_pulls.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.Log.Shale.LongPulls do 2 | @moduledoc false 3 | 4 | alias Bedrock.Internal.WaitingList 5 | 6 | @spec normalize_timeout_to_ms(term()) :: pos_integer() 7 | def normalize_timeout_to_ms(n), do: n |> normalize_timeout() |> max(10) |> min(10_000) 8 | 9 | @spec normalize_timeout(term()) :: integer() 10 | defp normalize_timeout(n) when is_integer(n), do: n 11 | defp normalize_timeout(_), do: 5000 12 | 13 | @spec notify_waiting_pullers(WaitingList.t(), Bedrock.version(), Bedrock.transaction()) :: 14 | WaitingList.t() 15 | def notify_waiting_pullers(waiting_pullers, version, transaction) do 16 | {new_map, entries} = WaitingList.remove_all(waiting_pullers, version) 17 | 18 | Enum.each(entries, fn {_deadline, reply_to_fn, _opts} -> 19 | reply_to_fn.({:ok, [transaction]}) 20 | end) 21 | 22 | new_map 23 | end 24 | 25 | @spec try_to_add_to_waiting_pullers( 26 | waiting_pullers :: WaitingList.t(), 27 | monotonic_now :: integer(), 28 | reply_to_fn :: (any() -> :ok), 29 | from_version :: Bedrock.version(), 30 | opts :: keyword() 31 | ) :: 32 | {:error, :version_too_new} | {:ok, updated_waiting_pullers :: WaitingList.t()} 33 | def try_to_add_to_waiting_pullers(waiting_pullers, _monotonic_now, reply_to_fn, from_version, opts) do 34 | {timeout_in_ms, opts} = Keyword.pop(opts, :willing_to_wait_in_ms) 35 | 36 | if timeout_in_ms == nil do 37 | {:error, :version_too_new} 38 | else 39 | timeout_ms = normalize_timeout_to_ms(timeout_in_ms) 40 | 41 | {new_waiting_pullers, _timeout} = 42 | WaitingList.insert(waiting_pullers, from_version, opts, reply_to_fn, timeout_ms) 43 | 44 | {:ok, new_waiting_pullers} 45 | end 46 | end 47 | 48 | @spec process_expired_deadlines_for_waiting_pullers( 49 | waiting_pullers :: WaitingList.t(), 50 | _monotic_now :: integer() 51 | ) :: WaitingList.t() 52 | def process_expired_deadlines_for_waiting_pullers(waiting_pullers, _monotic_now) do 53 | {new_waiting_pullers, expired_entries} = WaitingList.expire(waiting_pullers) 54 | 55 | WaitingList.reply_to_expired(expired_entries, {:ok, []}) 56 | 57 | new_waiting_pullers 58 | end 59 | 60 | @spec determine_timeout_for_next_puller_deadline(WaitingList.t(), integer()) :: 61 | pos_integer() | nil 62 | def determine_timeout_for_next_puller_deadline(waiting_pullers, now) do 63 | case WaitingList.next_timeout(waiting_pullers, fn -> now end) do 64 | :infinity -> nil 65 | timeout -> max(1, timeout) 66 | end 67 | end 68 | end 69 | -------------------------------------------------------------------------------- /test/bedrock/data_plane/log/shale/simple_pull_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.Log.Shale.SimplePullTest do 2 | use ExUnit.Case, async: false 3 | 4 | alias Bedrock.DataPlane.Log 5 | alias Bedrock.DataPlane.Log.Shale.Server 6 | alias Bedrock.DataPlane.Version 7 | 8 | @moduletag :tmp_dir 9 | 10 | setup %{tmp_dir: tmp_dir} do 11 | cluster = Bedrock.Cluster 12 | otp_name = :"simple_test_#{System.unique_integer([:positive])}" 13 | id = "simple_log_#{System.unique_integer([:positive])}" 14 | foreman = self() 15 | path = Path.join(tmp_dir, "log_segments") 16 | 17 | File.mkdir_p!(path) 18 | 19 | # Start the Shale server 20 | {:ok, pid} = 21 | GenServer.start_link( 22 | Server, 23 | {cluster, otp_name, id, foreman, path, true}, 24 | name: otp_name 25 | ) 26 | 27 | # Wait for initialization to complete 28 | Process.sleep(10) 29 | 30 | on_exit(fn -> 31 | if Process.alive?(pid) do 32 | GenServer.stop(pid) 33 | end 34 | end) 35 | 36 | {:ok, log: pid, path: path} 37 | end 38 | 39 | test "server starts and can respond to info", %{log: log} do 40 | # Just test that the server is alive and responsive 41 | assert {:ok, %{last_version: _}} = Log.info(log, [:last_version]) 42 | end 43 | 44 | test "boundary condition test - pull at exactly last_version", %{log: log} do 45 | # Get the initial last_version 46 | {:ok, initial_info} = Log.info(log, [:last_version]) 47 | last_version = initial_info.last_version 48 | 49 | # Pull from current last_version should return empty immediately (correct semantics) 50 | start_time = System.monotonic_time(:millisecond) 51 | result = Log.pull(log, last_version, timeout_in_ms: 100) 52 | end_time = System.monotonic_time(:millisecond) 53 | elapsed = end_time - start_time 54 | 55 | # Should return error immediately when no willing_to_wait specified (correct semantics) 56 | assert {:error, :version_too_new} = result 57 | assert elapsed < 50, "Should return immediately, but took #{elapsed}ms" 58 | end 59 | 60 | test "crash fix verification - no crash on empty log", %{log: log} do 61 | # This test verifies that empty log pulls don't crash 62 | # Previously this would crash with KeyError when active_segment was nil 63 | 64 | # Both pull scenarios should return error immediately when no willing_to_wait 65 | version_0 = Version.from_integer(0) 66 | assert {:error, :version_too_new} = Log.pull(log, version_0, timeout_in_ms: 100) 67 | assert {:error, :version_too_new} = Log.pull(log, version_0, []) 68 | 69 | # The key verification is that we reach here without crashing 70 | end 71 | end 72 | -------------------------------------------------------------------------------- /test/bedrock/data_plane/log/shale/transaction_streams_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.Log.Shale.TransactionStreamsTest do 2 | use ExUnit.Case, async: true 3 | 4 | alias Bedrock.DataPlane.Log.Shale.Segment 5 | alias Bedrock.DataPlane.Log.Shale.TransactionStreams 6 | alias Bedrock.DataPlane.Version 7 | alias Bedrock.Test.DataPlane.TransactionTestSupport 8 | 9 | # Helper functions for common test setup 10 | defp create_test_transaction(version, data) do 11 | TransactionTestSupport.new_log_transaction(Version.from_integer(version), data) 12 | end 13 | 14 | defp create_test_segment(path, min_version, transactions \\ nil) do 15 | %Segment{ 16 | path: path, 17 | min_version: Version.from_integer(min_version), 18 | transactions: transactions 19 | } 20 | end 21 | 22 | describe "TransactionStreams.from_segments/2 with unloaded segments" do 23 | test "handles nil transactions gracefully without enumerable protocol errors" do 24 | # Create a segment with nil transactions (simulating unloaded state) 25 | segment = create_test_segment("nonexistent_path_for_test", 1) 26 | 27 | # Before the fix, this would crash with: 28 | # "protocol Enumerable not implemented for type Atom. Got value: nil" 29 | # 30 | # After the fix, it should handle nil gracefully by calling ensure_transactions_are_loaded 31 | # The file doesn't exist, so we expect a File.Error, not an Enumerable error 32 | 33 | assert_raise File.Error, fn -> 34 | TransactionStreams.from_segments([segment], Version.from_integer(1)) 35 | end 36 | 37 | # The key point is that we get a File.Error (expected) rather than: 38 | # Protocol.UndefinedError with "protocol Enumerable not implemented for type Atom" 39 | end 40 | 41 | test "processes segments with pre-loaded transactions correctly" do 42 | # Create a segment with pre-loaded transactions (reversed order as stored) 43 | transaction_1 = create_test_transaction(1, %{"key1" => "value1"}) 44 | transaction_2 = create_test_transaction(2, %{"key2" => "value2"}) 45 | 46 | segment = create_test_segment("test_path", 1, [transaction_2, transaction_1]) 47 | 48 | # This should work normally 49 | assert {:ok, stream} = TransactionStreams.from_segments([segment], Version.from_integer(1)) 50 | 51 | # Convert stream to list to verify content - should get remaining transactions after target version 52 | # Since target_version=1 matches the first transaction, we get the rest 53 | assert [transaction] = Enum.to_list(stream) 54 | assert TransactionTestSupport.extract_log_version(transaction) == Version.from_integer(2) 55 | end 56 | end 57 | end 58 | -------------------------------------------------------------------------------- /lib/bedrock/control_plane/config/parameters.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.ControlPlane.Config.Parameters do 2 | @moduledoc """ 3 | A `Parameters` is a data structure that describes the t that are used 4 | to configure the cluster. 5 | """ 6 | 7 | @typedoc """ 8 | Struct representing the parameters that are used to configure the cluster. 9 | 10 | ## Fields: 11 | - `nodes` - A list of nodes that are participating in the cluster. 12 | - `ping_rate_in_hz` - The rate at which the director is to ping the nodes, 13 | expressed in Hertz. 14 | - `retransmission_rate_in_hz` - The rate at which the system is to retransmit 15 | messages, expressed in Hertz. 16 | - `replication_factor` - The (minimum) number of nodes that must acknowledge a 17 | write before it is considered successful. 18 | - `desired_coordinators` - The number of coordinators that are to be made 19 | available within the system. 20 | - `desired_logs` - The number of transaction logs that are to be made 21 | available 22 | - `desired_read_version_proxies` - The number of get read version proxies 23 | that are to be made available as part of the transaction system. 24 | - `desired_commit_proxies` - The number of commit proxies that are to be made 25 | available as part of the transaction system. 26 | - `desired_transaction_resolvers` - The number of transaction resolvers that 27 | are to be made available as part of the transaction system. 28 | """ 29 | 30 | @type rate_in_hz :: pos_integer() 31 | @type replication_factor :: pos_integer() 32 | 33 | @type t :: %{ 34 | nodes: [node()], 35 | ping_rate_in_hz: rate_in_hz(), 36 | retransmission_rate_in_hz: rate_in_hz(), 37 | desired_replication_factor: replication_factor(), 38 | desired_coordinators: pos_integer(), 39 | desired_logs: pos_integer(), 40 | desired_read_version_proxies: pos_integer(), 41 | desired_commit_proxies: pos_integer(), 42 | transaction_window_in_ms: pos_integer() 43 | } 44 | 45 | @spec new(coordinators :: [node()]) :: t() 46 | # 47 | def new(coordinators), 48 | do: %{ 49 | nodes: coordinators, 50 | desired_coordinators: length(coordinators), 51 | ping_rate_in_hz: 10, 52 | retransmission_rate_in_hz: 20, 53 | desired_replication_factor: 1, 54 | desired_logs: 1, 55 | desired_read_version_proxies: 1, 56 | desired_commit_proxies: 1, 57 | transaction_window_in_ms: 5_000 58 | } 59 | 60 | @spec put_desired_replication_factor(t(), replication_factor()) :: t() 61 | def put_desired_replication_factor(t, replication_factor), do: %{t | desired_replication_factor: replication_factor} 62 | end 63 | -------------------------------------------------------------------------------- /lib/bedrock/key.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.Key do 2 | @moduledoc """ 3 | Utilities for working with binary keys in Bedrock. 4 | 5 | Binary keys in Bedrock are ordered lexicographically, and this module provides 6 | utilities for key manipulation, ordering operations, and tuple packing/unpacking. 7 | 8 | ## Tuple Packing 9 | 10 | This module provides tuple packing and unpacking functionality that allows 11 | complex data structures (tuples, integers, floats, binaries, nil) to be 12 | encoded as binary keys that maintain lexicographic ordering. 13 | """ 14 | 15 | @type t :: binary() 16 | 17 | @doc """ 18 | Returns the next possible key after the given key in lexicographic order. 19 | 20 | This is useful for creating exclusive upper bounds in range operations. 21 | For example, if you want all keys that start with "prefix", you would 22 | use the range from "prefix" to `key_after("prefix")`. 23 | 24 | ## Examples 25 | 26 | iex> Bedrock.Key.key_after("abc") 27 | "abc\\0" 28 | 29 | iex> Bedrock.Key.key_after("") 30 | "\\0" 31 | 32 | """ 33 | @spec key_after(Bedrock.key()) :: binary() 34 | def key_after(key) when is_binary(key), do: key <> <<0>> 35 | 36 | def to_range(key) when is_binary(key), do: {key, key_after(key)} 37 | 38 | @doc """ 39 | Returns the lexicographically next key after the given key by incrementing 40 | the last non-0xFF byte. 41 | 42 | This is useful for creating tight upper bounds in prefix ranges. Unlike 43 | `key_after/1` which appends a null byte, `strinc/1` creates the minimal 44 | lexicographically next key. 45 | 46 | ## Examples 47 | 48 | iex> Bedrock.Key.strinc("abc") 49 | "abd" 50 | 51 | iex> Bedrock.Key.strinc(<<0, 1, 2>>) 52 | <<0, 1, 3>> 53 | iex> Bedrock.Key.strinc("hello") 54 | "hellp" 55 | 56 | ## Errors 57 | 58 | Raises an `ArgumentError` if the key contains only 0xFF bytes. 59 | 60 | iex> Bedrock.Key.strinc(<<0xFF, 0xFF>>) 61 | ** (ArgumentError) Key must contain at least one byte not equal to 0xFF 62 | 63 | """ 64 | @spec strinc(binary()) :: binary() 65 | def strinc(key) when is_binary(key) do 66 | prefix = rstrip_ff(key) 67 | prefix_len = byte_size(prefix) 68 | head = binary_part(prefix, 0, prefix_len - 1) 69 | tail = :binary.at(prefix, prefix_len - 1) 70 | head <> <> 71 | end 72 | 73 | defp rstrip_ff(<<>>), do: raise(ArgumentError, "Key must contain at least one byte not equal to 0xFF") 74 | 75 | defp rstrip_ff(key) do 76 | key_len = byte_size(key) 77 | 78 | case :binary.at(key, key_len - 1) do 79 | 0xFF -> key |> binary_part(0, key_len - 1) |> rstrip_ff() 80 | _ -> key 81 | end 82 | end 83 | end 84 | -------------------------------------------------------------------------------- /test/bedrock/data_plane/storage/basalt_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.Storage.BasaltTest do 2 | use ExUnit.Case, async: true 3 | 4 | import Bedrock.Test.Common.GenServerTestHelpers 5 | 6 | alias Bedrock.DataPlane.Storage.Basalt 7 | 8 | def id, do: Faker.UUID.v4() 9 | 10 | describe "Basalt.child_spec/1" do 11 | @tag :tmp_dir 12 | test "properly constructs a child spec", %{tmp_dir: tmp_dir} do 13 | expected_id = id() 14 | 15 | child_spec = 16 | Basalt.child_spec( 17 | cluster: "test", 18 | path: tmp_dir, 19 | id: expected_id, 20 | otp_name: :test_storage_engine, 21 | foreman: self() 22 | ) 23 | 24 | assert %{ 25 | id: {Basalt.Server, ^expected_id}, 26 | start: { 27 | GenServer, 28 | :start_link, 29 | [ 30 | Basalt.Server, 31 | {:test_storage_engine, pid, ^expected_id, ^tmp_dir}, 32 | [name: :test_storage_engine] 33 | ] 34 | } 35 | } = child_spec 36 | 37 | assert(is_pid(pid)) 38 | end 39 | end 40 | 41 | describe "Basalt" do 42 | @tag :tmp_dir 43 | test "lifecycle functions properly", %{tmp_dir: tmp_dir} do 44 | expected_id = id() 45 | 46 | child_spec = 47 | Basalt.child_spec( 48 | cluster: "test", 49 | path: tmp_dir, 50 | id: expected_id, 51 | otp_name: :test_storage_engine, 52 | foreman: self() 53 | ) 54 | 55 | pid = start_supervised!(child_spec) 56 | 57 | assert Process.alive?(pid) 58 | 59 | # We expect the worker to eventually send a message to the director 60 | # when it's ready to accept requests. 61 | assert_cast_received({:worker_health, ^expected_id, {:ok, pid}}, 5_000) do 62 | assert is_pid(pid) 63 | end 64 | 65 | # At this point, a physical database should have been created on disk. 66 | assert File.exists?(Path.join(tmp_dir, "dets")) 67 | end 68 | end 69 | 70 | describe "Basalt.Logic" do 71 | @tag :tmp_dir 72 | test "info/2 will return data for all topics it advertises as supported", %{tmp_dir: tmp_dir} do 73 | {:ok, state} = 74 | Basalt.Logic.startup( 75 | :test_storage_engine, 76 | self(), 77 | id(), 78 | tmp_dir 79 | ) 80 | 81 | {:ok, supported_info} = Basalt.Logic.info(state, :supported_info) 82 | 83 | assert {:ok, all_supported_info} = Basalt.Logic.info(state, supported_info) 84 | 85 | assert Enum.sort(Map.keys(all_supported_info)) == Enum.sort(supported_info) 86 | end 87 | end 88 | end 89 | -------------------------------------------------------------------------------- /test/bedrock/data_plane/transaction_atomic_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.DataPlane.TransactionAtomicTest do 2 | use ExUnit.Case, async: true 3 | 4 | alias Bedrock.DataPlane.Transaction 5 | 6 | # Helper functions for common test patterns 7 | defp create_transaction(mutations) do 8 | %{ 9 | mutations: mutations, 10 | read_conflicts: {nil, []}, 11 | write_conflicts: [] 12 | } 13 | end 14 | 15 | defp assert_roundtrip_encoding(transaction, expected_mutations) do 16 | encoded = Transaction.encode(transaction) 17 | assert {:ok, %{mutations: ^expected_mutations}} = Transaction.decode(encoded) 18 | end 19 | 20 | describe "atomic operations encoding/decoding" do 21 | test "encodes and decodes add with variable-length values" do 22 | mutations = [ 23 | # 5 as 2-byte little-endian 24 | {:atomic, :add, "counter", <<5, 0>>}, 25 | # 1 as 1-byte 26 | {:atomic, :add, "small_counter", <<1>>} 27 | ] 28 | 29 | transaction = create_transaction(mutations) 30 | assert_roundtrip_encoding(transaction, mutations) 31 | end 32 | 33 | test "encodes and decodes min with variable-length values" do 34 | mutations = [ 35 | # 10 as 3-byte little-endian 36 | {:atomic, :min, "min_val", <<10, 0, 0>>}, 37 | # 255 as 1-byte 38 | {:atomic, :min, "other", <<255>>} 39 | ] 40 | 41 | transaction = create_transaction(mutations) 42 | assert_roundtrip_encoding(transaction, mutations) 43 | end 44 | 45 | test "encodes and decodes max with variable-length values" do 46 | mutations = [ 47 | # 256 as 2-byte little-endian 48 | {:atomic, :max, "max_val", <<0, 1>>}, 49 | # 42 as 1-byte 50 | {:atomic, :max, "single", <<42>>} 51 | ] 52 | 53 | transaction = create_transaction(mutations) 54 | assert_roundtrip_encoding(transaction, mutations) 55 | end 56 | 57 | test "encodes and decodes mixed mutations including atomic operations" do 58 | mutations = [ 59 | {:set, "key1", "value1"}, 60 | {:atomic, :add, "counter", <<1, 0>>}, 61 | {:clear, "key2"}, 62 | {:atomic, :min, "min_val", <<5>>}, 63 | {:clear_range, "range_start", "range_end"}, 64 | {:atomic, :max, "max_val", <<100, 0, 1>>} 65 | ] 66 | 67 | transaction = create_transaction(mutations) 68 | assert_roundtrip_encoding(transaction, mutations) 69 | end 70 | 71 | test "handles empty atomic operands" do 72 | mutations = [ 73 | # Empty operand 74 | {:atomic, :add, "key", <<>>} 75 | ] 76 | 77 | transaction = create_transaction(mutations) 78 | assert_roundtrip_encoding(transaction, mutations) 79 | end 80 | end 81 | end 82 | -------------------------------------------------------------------------------- /lib/bedrock/cluster/gateway.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.Cluster.Gateway do 2 | @moduledoc """ 3 | The `Bedrock.Cluster.Gateway` is the interface to a GenServer that is 4 | responsible for finding and holding the current coordinator and director for 5 | a cluster. 6 | """ 7 | 8 | use Bedrock.Internal.GenServerApi, for: __MODULE__.Server 9 | 10 | @type ref :: pid() | atom() | {atom(), node()} 11 | 12 | @spec begin_transaction( 13 | gateway_ref :: ref(), 14 | opts :: [ 15 | retry_count: pos_integer(), 16 | timeout_in_ms: Bedrock.timeout_in_ms() 17 | ] 18 | ) :: {:ok, Bedrock.Internal.Repo.transaction()} | {:error, :timeout} 19 | def begin_transaction(gateway, opts \\ []), 20 | do: call(gateway, {:begin_transaction, opts}, opts[:timeout_in_ms] || :infinity) 21 | 22 | @doc """ 23 | Renew the lease for a transaction based on the read version. 24 | """ 25 | @spec renew_read_version_lease( 26 | gateway_ref :: ref(), 27 | read_version :: Bedrock.version(), 28 | opts :: [ 29 | timeout_in_ms: Bedrock.timeout_in_ms() 30 | ] 31 | ) :: 32 | {:ok, lease_deadline_ms :: Bedrock.interval_in_ms()} | {:error, :lease_expired} 33 | def renew_read_version_lease(t, read_version, opts \\ []), 34 | do: call(t, {:renew_read_version_lease, read_version}, opts[:timeout_in_ms] || :infinity) 35 | 36 | @doc """ 37 | Report the addition of a new worker to the cluster director. It does so by 38 | sending an asynchronous message to the specified gateway process. The gateway 39 | process will then gather some information from the worker and pass it to the 40 | cluster director. 41 | 42 | ## Parameters 43 | - `gateway`: The GenServer name or PID of the gateway that will handle the 44 | new worker information. 45 | - `worker`: The PID of the new worker process that has been added. It will 46 | be interrogated for details before passing it to the cluster director. 47 | 48 | ## Returns 49 | - `:ok`: Always returns `:ok` as the message is sent asynchronously. 50 | """ 51 | @spec advertise_worker(gateway :: ref(), worker :: pid()) :: :ok 52 | def advertise_worker(gateway, worker), do: cast(gateway, {:advertise_worker, worker}) 53 | 54 | @doc """ 55 | Get the cluster descriptor from the gateway. 56 | This includes the coordinator nodes and other cluster configuration. 57 | """ 58 | @spec get_descriptor( 59 | gateway :: ref(), 60 | opts :: [timeout_in_ms: Bedrock.timeout_in_ms()] 61 | ) :: {:ok, Bedrock.Cluster.Descriptor.t()} | {:error, :unavailable | :timeout | :unknown} 62 | def get_descriptor(gateway, opts \\ []), do: call(gateway, :get_descriptor, opts[:timeout_in_ms] || 1000) 63 | end 64 | -------------------------------------------------------------------------------- /guides/quick-reads/recovery/log-replay.md: -------------------------------------------------------------------------------- 1 | # Log Replay: Data Migration to Trusted Infrastructure 2 | 3 | **Copying committed transactions from potentially compromised logs to verified new infrastructure.** 4 | 5 | The log replay phase transfers all committed transaction data from old [logs](../../deep-dives/architecture/data-plane/log.md) to newly recruited log services. Rather than attempting to salvage potentially compromised infrastructure, Bedrock systematically copies transaction data to verified, reliable storage before the new system begins operation. 6 | 7 | ## Core Process 8 | 9 | **Smart Pairing Algorithm** 10 | Recovery pairs each new log with old logs using round-robin distribution. When scaling from 2 to 4 logs: 11 | 12 | - New log 1 ← Old log 1 13 | - New log 2 ← Old log 2 14 | - New log 3 ← Old log 1 (cycles back) 15 | - New log 4 ← Old log 2 16 | 17 | **Selective Data Migration** 18 | Only committed transactions within the established [version](../../glossary.md#version) range determined during [version determination](version-determination.md) undergo copying. Uncommitted transactions are deliberately excluded to maintain system consistency. 19 | 20 | **Parallel Execution** 21 | Each old-to-new log pairing operates in parallel, maximizing throughput while maintaining strict consistency requirements. 22 | 23 | ## Reliability Philosophy 24 | 25 | Bedrock chooses data integrity over operational efficiency. Even though old logs contain correct transaction data, recovery copies this information to newly recruited logs rather than reusing potentially compromised storage. This ensures all transaction data resides on verified infrastructure before system startup. 26 | 27 | The new [transaction system layout](transaction-system-layout.md) often differs significantly—different log services, node assignments, and potentially different durability policies. Complete data migration ensures compatibility between system generations. 28 | 29 | ## Error Handling 30 | 31 | - **Service Failures**: Controlled stall with detailed diagnostics 32 | - **Epoch Conflicts**: Immediate termination if `:newer_epoch_exists` indicates a newer recovery attempt 33 | - **Fail-Fast**: Clean termination enables coordinator restart with fresh state 34 | 35 | ## Phase Integration 36 | 37 | **Prerequisites**: [Log recruitment](log-recruitment.md), [version determination](version-determination.md), [service locking](service-locking.md) 38 | **Next Phase**: [Sequencer startup](sequencer-startup.md) with populated transaction data 39 | **Implementation**: `lib/bedrock/control_plane/director/recovery/log_replay_phase.ex` 40 | 41 | This phase marks the transformation from architectural planning to operational infrastructure capable of serving transaction workloads with complete data integrity confidence. 42 | -------------------------------------------------------------------------------- /test/bedrock/control_plane/coordinator/integration_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.ControlPlane.Coordinator.IntegrationTest do 2 | use ExUnit.Case, async: true 3 | 4 | alias Bedrock.ControlPlane.Coordinator.Commands 5 | alias Bedrock.ControlPlane.Coordinator.State 6 | 7 | # Helper functions for test setup 8 | defp build_state(opts) do 9 | defaults = [ 10 | leader_node: Node.self(), 11 | my_node: Node.self(), 12 | epoch: 1 13 | ] 14 | 15 | opts = Keyword.merge(defaults, opts) 16 | struct!(State, opts) 17 | end 18 | 19 | defp simulate_ping_response(%State{} = state) do 20 | leader = if state.leader_node == state.my_node, do: self() 21 | {:pong, state.epoch, leader} 22 | end 23 | 24 | describe "service registration integration" do 25 | test "handles register_services call" do 26 | services = [{"test_service", :storage, {:worker, :node@host}}] 27 | 28 | # Test that the command is properly formed 29 | command = Commands.register_services(services) 30 | assert {:register_services, %{services: ^services}} = command 31 | end 32 | 33 | test "handles deregister_services call" do 34 | service_ids = ["existing_service"] 35 | 36 | # Test that the command is properly formed 37 | command = Commands.deregister_services(service_ids) 38 | assert {:deregister_services, %{service_ids: ^service_ids}} = command 39 | end 40 | 41 | test "ping handler returns correct format when leader" do 42 | epoch = 42 43 | leader_pid = self() 44 | 45 | state = build_state(leader_node: Node.self(), epoch: epoch) 46 | response = simulate_ping_response(state) 47 | 48 | assert {:pong, ^epoch, ^leader_pid} = response 49 | end 50 | 51 | test "ping handler returns nil leader when not leader" do 52 | epoch = 42 53 | 54 | state = build_state(leader_node: :other_node, epoch: epoch) 55 | response = simulate_ping_response(state) 56 | 57 | assert {:pong, ^epoch, nil} = response 58 | end 59 | end 60 | 61 | describe "director startup integration" do 62 | test "director receives service directory at startup" do 63 | services = %{ 64 | "service_1" => {:storage, {:worker1, :node1@host}}, 65 | "service_2" => {:log, {:worker2, :node2@host}} 66 | } 67 | 68 | # Verify that the director startup args would include the services 69 | expected_args = [ 70 | cluster: TestCluster, 71 | config: %{}, 72 | old_transaction_system_layout: %{}, 73 | epoch: 1, 74 | coordinator: self(), 75 | services: services 76 | ] 77 | 78 | # Verify services are correctly included in startup args 79 | assert expected_args[:services] == services 80 | end 81 | end 82 | end 83 | -------------------------------------------------------------------------------- /lib/bedrock/control_plane/director/recovery/tsl_validation_phase.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.ControlPlane.Director.Recovery.TSLValidationPhase do 2 | @moduledoc """ 3 | Early recovery phase that validates TSL type safety to prevent data corruption. 4 | 5 | This phase runs defensively on recovered TSL data, checking for type mismatches 6 | like integer-to-binary version conversion errors that can cause MVCC lookup failures. 7 | It specifically validates: 8 | 9 | - `logs` field has integer ranges (not Version.t() binaries) 10 | - `version_vector` field contains Version.t() binaries (not integers) 11 | - `durable_version` field contains Version.t() binary (not integer) 12 | - `storage_teams` structure is valid 13 | - `resolvers` structure is valid 14 | 15 | ## Error Handling 16 | 17 | On validation failure, this phase stalls recovery with `{:corrupted_tsl, details}` 18 | to allow operators to investigate and fix the underlying data corruption rather 19 | than failing silently or propagating errors further into the recovery process. 20 | 21 | ## Integration Point 22 | 23 | Should run early in the recovery pipeline after TSL data is loaded but before 24 | any processing that depends on type-correct TSL fields. This provides a clear 25 | failure point with detailed diagnostics. 26 | 27 | Transitions to the next appropriate recovery phase on successful validation. 28 | """ 29 | 30 | use Bedrock.ControlPlane.Director.Recovery.RecoveryPhase 31 | 32 | import Bedrock.ControlPlane.Director.Recovery.Telemetry 33 | 34 | alias Bedrock.ControlPlane.Config.TSLTypeValidator 35 | 36 | @doc """ 37 | Validates TSL type safety using defensive validation. 38 | 39 | Returns `{:stalled, {:corrupted_tsl, validation_error}}` on validation failure 40 | to halt recovery and provide clear diagnostics. Logs detailed error information 41 | for debugging the underlying data corruption. 42 | 43 | On success, transitions to the next recovery phase without modifying the 44 | recovery attempt (this is a pure validation phase). 45 | """ 46 | @impl true 47 | def execute(%RecoveryAttempt{} = recovery_attempt, %{old_transaction_system_layout: %{} = tsl_to_validate}) do 48 | case TSLTypeValidator.validate_type_safety(tsl_to_validate) do 49 | :ok -> 50 | trace_recovery_tsl_validation_success() 51 | {recovery_attempt, Bedrock.ControlPlane.Director.Recovery.LockingPhase} 52 | 53 | {:error, validation_error} -> 54 | trace_recovery_tsl_validation_failed(tsl_to_validate, validation_error) 55 | {recovery_attempt, {:stalled, {:corrupted_tsl, validation_error}}} 56 | end 57 | end 58 | 59 | def execute(recovery_attempt, _context), 60 | do: {recovery_attempt, Bedrock.ControlPlane.Director.Recovery.InitializationPhase} 61 | end 62 | -------------------------------------------------------------------------------- /lib/bedrock/exceptions.ex: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.StorageError do 2 | @moduledoc """ 3 | Exception raised when a storage operation fails with a retryable error. 4 | 5 | These errors indicate temporary storage unavailability or timeouts that 6 | should trigger transaction retry at the TransactionManager level. 7 | 8 | Common reasons: 9 | - `:unavailable` - No storage servers available for the key range 10 | - `:timeout` - Storage operation timed out 11 | 12 | These are exceptional conditions that interrupt normal transaction flow 13 | but can potentially succeed on retry. 14 | """ 15 | defexception [:reason, :operation, :key, :message] 16 | 17 | @impl true 18 | def exception(opts) do 19 | reason = Keyword.fetch!(opts, :reason) 20 | operation = Keyword.get(opts, :operation) 21 | key = Keyword.get(opts, :key) 22 | 23 | message = build_message(reason, operation, key) 24 | 25 | %__MODULE__{ 26 | reason: reason, 27 | operation: operation, 28 | key: key, 29 | message: message 30 | } 31 | end 32 | 33 | defp build_message(reason, operation, key) do 34 | base = "Storage error: #{inspect(reason)}" 35 | 36 | with_op = if operation, do: "#{base} during #{operation}", else: base 37 | 38 | if key do 39 | "#{with_op} for key #{inspect(key)}" 40 | else 41 | with_op 42 | end 43 | end 44 | end 45 | 46 | defmodule Bedrock.TransactionError do 47 | @moduledoc """ 48 | Exception raised when a transaction operation fails with a non-retryable error. 49 | 50 | These errors indicate programming errors or invalid operations that will 51 | not succeed on retry and should fail immediately. 52 | 53 | Common reasons: 54 | - Invalid key format 55 | - Decode errors 56 | - Version conflicts that indicate logic errors 57 | - Other unexpected error conditions 58 | 59 | These errors should not trigger transaction retry as they represent 60 | fundamental problems that need to be fixed in the calling code. 61 | """ 62 | defexception [:reason, :operation, :key, :message] 63 | 64 | @impl true 65 | def exception(opts) do 66 | reason = Keyword.fetch!(opts, :reason) 67 | operation = Keyword.get(opts, :operation) 68 | key = Keyword.get(opts, :key) 69 | 70 | message = build_message(reason, operation, key) 71 | 72 | %__MODULE__{ 73 | reason: reason, 74 | operation: operation, 75 | key: key, 76 | message: message 77 | } 78 | end 79 | 80 | defp build_message(reason, operation, key) do 81 | base = "Transaction error: #{inspect(reason)}" 82 | 83 | with_op = if operation, do: "#{base} during #{operation}", else: base 84 | 85 | if key do 86 | "#{with_op} for key #{inspect(key)}" 87 | else 88 | with_op 89 | end 90 | end 91 | end 92 | -------------------------------------------------------------------------------- /guides/quick-reads/recovery/transaction-system-layout.md: -------------------------------------------------------------------------------- 1 | # Transaction System Layout Phase 2 | 3 | **Recovery phase 12: Creating the coordination blueprint that enables distributed transaction processing.** 4 | 5 | After recovery brings components online individually, they exist in isolation—the [sequencer](../../deep-dives/architecture/data-plane/sequencer.md) can't notify [logs](../../deep-dives/architecture/data-plane/log.md), [proxies](../../deep-dives/architecture/data-plane/commit-proxy.md) can't route to [resolvers](../../deep-dives/architecture/data-plane/resolver.md), and [storage teams](../../deep-dives/architecture/data-plane/storage.md) operate independently. The Transaction System Layout (TSL) phase solves this coordination problem. 6 | 7 | ## What It Does 8 | 9 | This phase creates the authoritative coordination map that tells every component how to find and communicate with every other component it needs to work with. Think of it as building the distributed system's phone book and organizational chart simultaneously. 10 | 11 | The TSL contains: 12 | 13 | - Process identifiers for all components 14 | - Key range assignments for storage teams and resolvers 15 | - Log shard mappings 16 | - Communication endpoints and coordination relationships 17 | 18 | ## The Process 19 | 20 | **Validation**: Quick health check confirms all components remain operational since startup. 21 | 22 | **Blueprint Assembly**: Creates the complete [Transaction System Layout](../../quick-reads/transaction-system-layout.md) data structure with all coordination mappings. 23 | 24 | **Pipeline Test**: Stores the TSL using a full system transaction that exercises the entire transaction processing pipeline—if any component can't coordinate properly, the transaction fails and recovery terminates. 25 | 26 | **Service Unlock Preparation**: Configures all services for coordinated unlocking once the TSL is successfully persisted. 27 | 28 | ## Critical Safety 29 | 30 | The system transaction that persists the TSL serves as the ultimate validation—if components can't coordinate well enough to complete this transaction, they can't handle normal operations. Any failure terminates recovery immediately rather than deploying a broken system. 31 | 32 | ## Next Steps 33 | 34 | Success transitions to [Persistence](persistence.md), which handles the final storage of configuration data and prepares for [Monitoring](monitoring.md) establishment. 35 | 36 | ## Implementation 37 | 38 | **Source**: `lib/bedrock/control_plane/director/recovery/transaction_system_layout_phase.ex` 39 | 40 | ## See Also 41 | 42 | - [Transaction System Layout](../../quick-reads/transaction-system-layout.md) - The coordination blueprint structure 43 | - [Recovery](../recovery.md) - Complete recovery process overview 44 | - [Persistence](persistence.md) - Next recovery phase 45 | -------------------------------------------------------------------------------- /test/bedrock/cluster/gateway/transaction_builder/no_write_conflict_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.Cluster.Gateway.TransactionBuilder.NoWriteConflictTest do 2 | use ExUnit.Case, async: true 3 | 4 | alias Bedrock.Cluster.Gateway.TransactionBuilder.Tx 5 | alias Bedrock.DataPlane.Transaction 6 | 7 | # Helper function to build and decode transaction 8 | defp build_and_decode_transaction(operations_fn) do 9 | {:ok, result} = 10 | Tx.new() 11 | |> operations_fn.() 12 | |> Tx.commit(nil) 13 | |> Transaction.decode() 14 | 15 | result 16 | end 17 | 18 | # Helper to extract single keys from write conflict ranges 19 | defp extract_conflict_keys(write_conflicts) do 20 | Enum.flat_map(write_conflicts, fn {start_key, end_key} -> 21 | if end_key == start_key <> <<0>>, do: [start_key], else: [] 22 | end) 23 | end 24 | 25 | describe "no_write_conflict option" do 26 | test "no_write_conflict prevents write conflicts from being created" do 27 | # Use pattern matching to extract and assert fields in one step 28 | assert %{ 29 | mutations: [ 30 | {:set, "test1", "value1"}, 31 | {:set, "test2", "value2"}, 32 | {:atomic, :add, "counter1", "\x01"}, 33 | {:atomic, :add, "counter2", "\x01"} 34 | ], 35 | write_conflicts: write_conflicts 36 | } = 37 | build_and_decode_transaction(fn tx -> 38 | tx 39 | |> Tx.set("test1", "value1") 40 | |> Tx.set("test2", "value2", no_write_conflict: true) 41 | |> Tx.atomic_operation("counter1", :add, <<1>>) 42 | |> Tx.atomic_operation("counter2", :add, <<1>>) 43 | end) 44 | 45 | # Only operations without no_write_conflict should create write conflicts 46 | assert ["test1"] = extract_conflict_keys(write_conflicts) 47 | end 48 | 49 | test "operations always remove conflicting operations regardless of no_write_conflict" do 50 | # Pattern match to assert mutations directly 51 | assert %{mutations: [{:set, "key", "value2"}]} = 52 | build_and_decode_transaction(fn tx -> 53 | tx 54 | |> Tx.set("key", "value1") 55 | |> Tx.set("key", "value2", no_write_conflict: true) 56 | end) 57 | end 58 | 59 | test "regular operations remove conflicting operations" do 60 | # Pattern match to assert mutations directly 61 | assert %{mutations: [{:set, "key", "value2"}]} = 62 | build_and_decode_transaction(fn tx -> 63 | tx 64 | |> Tx.set("key", "value1") 65 | |> Tx.set("key", "value2") 66 | end) 67 | end 68 | end 69 | end 70 | -------------------------------------------------------------------------------- /guides/quick-reads/transaction-system-layout.md: -------------------------------------------------------------------------------- 1 | # Transaction System Layout 2 | 3 | **The coordination blueprint that enables distributed transaction processing.** 4 | 5 | After recovery brings individual components online, they need to know who handles what. The Transaction System Layout (TSL) provides this coordination map—without it, the sequencer can't notify logs, proxies can't route to resolvers, and storage teams operate in isolation. 6 | 7 | ## Core Function 8 | 9 | The TSL maps every component relationship needed for transaction processing. Think of it as the distributed system's phone book and organizational chart combined. 10 | 11 | ## Essential Components 12 | 13 | ### Process References 14 | 15 | - **Sequencer**: Global transaction ordering authority 16 | - **Rate Keeper**: System-wide rate limiting and flow control 17 | - **Commit Proxies**: Transaction entry points and batching coordinators 18 | - **Resolvers**: MVCC conflict detection by key range 19 | 20 | ### System Topology 21 | 22 | - **Logs**: Transaction persistence with shard-to-server mappings 23 | - **Storage Teams**: Data storage with key range assignments 24 | - **Services**: Complete worker registry with operational status 25 | 26 | ### Coordination Metadata 27 | 28 | - **Layout ID**: Unique identifier for this system configuration 29 | - **Epoch**: Recovery generation for change tracking 30 | - **Director**: Current cluster coordination authority 31 | 32 | ## When Created 33 | 34 | TSL construction happens during recovery phase 12—after all components are operational but before they can coordinate transactions. The layout is immediately persisted as the authoritative system configuration. 35 | 36 | ## Quick Reference 37 | 38 | ```elixir 39 | # View current layout 40 | {:ok, layout} = Bedrock.ControlPlane.Director.fetch_transaction_system_layout(director_pid) 41 | 42 | # Layout structure 43 | %TransactionSystemLayout{ 44 | id: 42, 45 | epoch: 5, 46 | director: #PID<0.123.45>, 47 | sequencer: #PID<0.124.46>, 48 | rate_keeper: #PID<0.126.48>, 49 | proxies: [#PID<0.125.47>, #PID<0.127.49>], 50 | resolvers: [ 51 | %{start_key: "", resolver: #PID<0.128.50>}, 52 | %{start_key: "m", resolver: #PID<0.129.51>} 53 | ], 54 | logs: %{0 => log_descriptor, 1 => log_descriptor}, 55 | storage_teams: [storage_team_descriptor, ...] 56 | } 57 | ``` 58 | 59 | ## Implementation 60 | 61 | - **Main Module**: `lib/bedrock/control_plane/config/transaction_system_layout.ex` 62 | - **Creation Phase**: `lib/bedrock/control_plane/director/recovery/transaction_system_layout_phase.ex` 63 | 64 | ## See Also 65 | 66 | - [Transaction System Layout Phase](recovery/transaction-system-layout.md) - Detailed creation process 67 | - [Recovery](recovery.md) - Recovery phase context 68 | - [Transactions](transactions.md) - How TSL enables transaction coordination 69 | -------------------------------------------------------------------------------- /test/bedrock/cluster/gateway/discovery_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Bedrock.Cluster.Gateway.DiscoveryTest do 2 | use ExUnit.Case, async: true 3 | 4 | alias Bedrock.Cluster.Gateway.Discovery 5 | alias Bedrock.Cluster.Gateway.State 6 | 7 | describe "leader discovery and selection" do 8 | test "select_leader_from_responses/1 chooses coordinator with highest epoch" do 9 | responses = [ 10 | {:node1, {:pong, 5, :coordinator_pid_1}}, 11 | {:node2, {:pong, 8, :coordinator_pid_2}}, 12 | {:node3, {:pong, 3, :coordinator_pid_3}} 13 | ] 14 | 15 | assert {:ok, {:coordinator_pid_2, 8}} = 16 | Discovery.select_leader_from_responses(responses) 17 | end 18 | 19 | test "select_leader_from_responses/1 filters out nil leaders" do 20 | responses = [ 21 | {:node1, {:pong, 5, nil}}, 22 | {:node2, {:pong, 8, :coordinator_pid_2}}, 23 | {:node3, {:pong, 10, nil}} 24 | ] 25 | 26 | assert {:ok, {:coordinator_pid_2, 8}} = 27 | Discovery.select_leader_from_responses(responses) 28 | end 29 | 30 | test "select_leader_from_responses/1 returns error when no leaders available" do 31 | responses = [ 32 | {:node1, {:pong, 5, nil}}, 33 | {:node2, {:pong, 8, nil}}, 34 | {:node3, {:pong, 3, nil}} 35 | ] 36 | 37 | assert Discovery.select_leader_from_responses(responses) == {:error, :unavailable} 38 | end 39 | 40 | test "select_leader_from_responses/1 handles empty responses" do 41 | assert Discovery.select_leader_from_responses([]) == {:error, :unavailable} 42 | end 43 | 44 | test "select_leader_from_responses/1 handles single leader" do 45 | responses = [ 46 | {:node1, {:pong, 5, :coordinator_pid_1}} 47 | ] 48 | 49 | assert {:ok, {:coordinator_pid_1, 5}} = 50 | Discovery.select_leader_from_responses(responses) 51 | end 52 | end 53 | 54 | describe "change_coordinator/2" do 55 | setup do 56 | state = %State{ 57 | node: :test_node, 58 | cluster: TestCluster, 59 | known_coordinator: :coordinator_ref 60 | } 61 | 62 | %{state: state} 63 | end 64 | 65 | test "does not change when coordinator is the same", %{state: state} do 66 | assert %State{known_coordinator: :coordinator_ref} = 67 | Discovery.change_coordinator(state, :coordinator_ref) 68 | end 69 | 70 | test "sets known_coordinator to unavailable when requested", %{state: state} do 71 | assert %State{known_coordinator: :unavailable} = 72 | Discovery.change_coordinator(state, :unavailable) 73 | end 74 | end 75 | 76 | # Note: try_direct_coordinator_call/2 would require proper mocking setup 77 | # Integration tests would be more appropriate for testing this functionality 78 | end 79 | --------------------------------------------------------------------------------