From 7e90431caa5afd94834b0262bf8c0a1c95a4a29f Mon Sep 17 00:00:00 2001 From: James Fish Date: Sun, 9 Apr 2017 17:57:26 +0100 Subject: [PATCH 01/20] Introduce DBConnection.StreamStage --- integration_test/cases/stream_stage_test.exs | 102 ++++++++++ integration_test/tests.exs | 1 + lib/db_connection.ex | 144 ++++++++++++++ lib/db_connection/stream_stage.ex | 189 +++++++++++++++++++ mix.exs | 2 + mix.lock | 2 + test/test_support.exs | 26 ++- 7 files changed, 462 insertions(+), 4 deletions(-) create mode 100644 integration_test/cases/stream_stage_test.exs create mode 100644 lib/db_connection/stream_stage.ex diff --git a/integration_test/cases/stream_stage_test.exs b/integration_test/cases/stream_stage_test.exs new file mode 100644 index 00000000..e2beabbc --- /dev/null +++ b/integration_test/cases/stream_stage_test.exs @@ -0,0 +1,102 @@ +defmodule StreamStageTest do + use ExUnit.Case, async: true + + alias TestPool, as: P + alias DBConnection.StreamStage, as: S + alias TestAgent, as: A + alias TestQuery, as: Q + alias TestCursor, as: C + alias TestResult, as: R + + test "stream returns result" do + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, %C{}, :newer_state}, + {:ok, %R{}, :newest_state}, + {:deallocate, %R{}, :state2}, + {:ok, :deallocated, :new_state2}, + {:ok, :commited, :newer_state2} + ] + {:ok, agent} = A.start_link(stack) + + opts = [agent: agent, parent: self()] + {:ok, pool} = P.start_link(opts) + {:ok, stage} = S.stream_link(pool, %Q{}, [:param], opts) + assert stage |> Flow.from_stage() |> Enum.to_list() == [%R{}, %R{}] + + GenStage.stop(stage) + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_declare: [%Q{}, [:param], _, :new_state], + handle_first: [%Q{}, %C{}, _, :newer_state], + handle_next: [%Q{}, %C{}, _, :newest_state], + handle_deallocate: [%Q{}, %C{}, _, :state2], + handle_commit: [_, :new_state2] + ] = A.record(agent) + end + + test "prepare_stream returns result" do + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, %Q{}, :newer_state}, + {:ok, %C{}, :newest_state}, + {:ok, %R{}, :state2}, + {:deallocate, %R{}, :new_state2}, + {:ok, :deallocated, :newer_state2}, + {:ok, :commited, :newest_state2} + ] + {:ok, agent} = A.start_link(stack) + + opts = [agent: agent, parent: self()] + {:ok, pool} = P.start_link(opts) + {:ok, stage} = S.prepare_stream_link(pool, %Q{}, [:param], opts) + assert stage |> Flow.from_stage() |> Enum.to_list() == [%R{}, %R{}] + + GenStage.stop(stage) + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_prepare: [%Q{}, _, :new_state], + handle_declare: [%Q{}, [:param], _, :newer_state], + handle_first: [%Q{}, %C{}, _, :newest_state], + handle_next: [%Q{}, %C{}, _, :state2], + handle_deallocate: [%Q{}, %C{}, _, :new_state2], + handle_commit: [_, :newer_state2] + ] = A.record(agent) + end + + test "stream can be rolled back in flat map" do + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, %C{}, :newer_state}, + {:ok, %R{}, :newest_state}, + {:ok, :deallocated, :state2}, + {:ok, :rolledback, :new_state2} + ] + {:ok, agent} = A.start_link(stack) + + opts = [agent: agent, parent: self()] + {:ok, pool} = P.start_link(opts) + map = fn(conn, _) -> P.rollback(conn, :normal) end + {:ok, stage} = S.stream_link(pool, %Q{}, [:param], [flat_map: map] ++ opts) + mon = Process.monitor(stage) + catch_exit(stage |> Flow.from_stage() |> Enum.to_list()) + + assert_receive {:DOWN, ^ref, :process, ^stage, :normal} + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_declare: [%Q{}, [:param], _, :new_state], + handle_first: [%Q{}, %C{}, _, :newer_state], + handle_deallocate: [%Q{}, %C{}, _, :newest_state], + handle_rollback: [_, :state2] + ] = A.record(agent) + end +end diff --git a/integration_test/tests.exs b/integration_test/tests.exs index f8ed4c1a..ada91813 100644 --- a/integration_test/tests.exs +++ b/integration_test/tests.exs @@ -9,6 +9,7 @@ Code.require_file "cases/prepare_execute_test.exs", __DIR__ Code.require_file "cases/prepare_stream_test.exs", __DIR__ Code.require_file "cases/prepare_test.exs", __DIR__ Code.require_file "cases/queue_test.exs", __DIR__ +Code.require_file "cases/stream_stage_test.exs", __DIR__ Code.require_file "cases/stream_test.exs", __DIR__ Code.require_file "cases/transaction_execute_test.exs", __DIR__ Code.require_file "cases/transaction_test.exs", __DIR__ diff --git a/lib/db_connection.ex b/lib/db_connection.ex index 4755e323..5feac0e9 100644 --- a/lib/db_connection.ex +++ b/lib/db_connection.ex @@ -913,6 +913,72 @@ defmodule DBConnection do resource(conn, start, &fetch/3, &deallocate/3, opts).(acc, fun) end + ## Stage Transaction + + @doc false + @spec stage_begin(pool :: GenServer.server, opts :: Keyword.t) :: t + def stage_begin(pool, opts) do + {result, log_info} = stage_begin_meter(pool, opts) + transaction_log(log_info) + case result do + {:ok, conn} -> + conn + {:raise, err} -> + raise err + {kind, reason, stack} -> + :erlang.raise(kind, reason, stack) + end + end + + @doc false + @spec stage_transaction(t, (t -> result), opts :: Keyword.t) :: + {:ok, result} | {:error, reason :: any} | :closed when result: var + def stage_transaction(conn, fun, opts) do + case get_info(conn) do + {:failed, _} -> + stage_rollback(conn, opts) + {:idle, _} -> + raise "not inside transaction" + {:transaction, _} -> + stage_nested(conn, fun, opts) + :closed -> + :closed + end + end + + @doc false + @spec stage_commit(t, Keyword.t) :: :ok | {:error, :rollback} + def stage_commit(conn, opts) do + log = Keyword.get(opts, :log) + {result, log_info} = stage_conclude(conn, &commit/4, log, opts, :ok) + transaction_log(log_info) + case result do + {:raise, err} -> + raise err + {kind, reason, stack} -> + :erlang.raise(kind, reason, stack) + other -> + other + end + end + + @doc false + @spec stage_rollback(t, Keyword.t) :: {:error, :rollback} + def stage_rollback(conn, opts) do + log = Keyword.get(opts, :log) + result = {:error, :rollback} + {result, log_info} = stage_conclude(conn, &rollback/4, log, opts, result) + transaction_log(log_info) + case result do + {:raise, err} -> + raise err + {kind, reason, stack} -> + :erlang.raise(kind, reason, stack) + other -> + other + end + end + ## Helpers defp checkout(pool, opts) do @@ -1512,6 +1578,84 @@ defmodule DBConnection do Stream.resource(start, next, stop) end + defp stage_begin_meter(pool, opts) do + case Keyword.get(opts, :log) do + nil -> + stage_begin_meter(pool, nil, [], opts) + log -> + times = [checkout: time()] + stage_begin_meter(pool, log, times, opts) + end + end + + defp stage_begin_meter(pool, log, times, opts) do + {conn, conn_state} = checkout(pool, opts) + put_info(conn, :idle, conn_state) + stage_begin_meter(conn, conn_state, log, times, opts) + end + + defp stage_begin_meter(conn, conn_state, nil, [], opts) do + case handle(conn, conn_state, :handle_begin, opts, :transaction) do + {:ok, _} -> + {{:ok, conn}, nil} + error -> + run_end(conn, opts) + {error, nil} + end + end + defp stage_begin_meter(conn, conn_state, log, times, opts) do + start = time() + result = handle(conn, conn_state, :handle_begin, opts, :transaction) + times = [stop: time(), start: start] ++ times + log_info = {log, times, :handle_begin, result} + case result do + {:ok, _} -> + stage_begin_log(conn, log_info, opts) + error -> + run_end(conn, opts) + {error, log_info} + end + end + + defp stage_begin_log(conn, {log, _, _, _} = log_info, opts) do + try do + transaction_log(log_info) + catch + kind, reason -> + result = {kind, reason, System.stacktrace()} + stage_conclude(conn, &rollback/4, log, opts, result) + else + _ -> + {{:ok, conn}, nil} + end + end + + defp stage_nested(conn, fun, opts) do + %DBConnection{conn_ref: conn_ref} = conn + try do + fun.(conn) + catch + :throw, {:rollback, ^conn_ref, reason} -> + stage_rollback(conn, opts) + {:error, reason} + kind, reason -> + stack = System.stacktrace() + stage_rollback(conn, opts) + :erlang.raise(kind, reason, stack) + else + result -> + {:ok, result} + end + end + + defp stage_conclude(conn, fun, log, opts, result) do + try do + fun.(conn, log, opts, result) + after + run_end(conn, opts) + end + end + defp put_info(conn, status, conn_state) do _ = Process.put(key(conn), {status, conn_state}) :ok diff --git a/lib/db_connection/stream_stage.ex b/lib/db_connection/stream_stage.ex new file mode 100644 index 00000000..9cbedcfa --- /dev/null +++ b/lib/db_connection/stream_stage.ex @@ -0,0 +1,189 @@ +defmodule DBConnection.StreamStage do + @moduledoc """ + A `GenStage` producer that streams the results of a query inside a + transaction. + """ + alias __MODULE__, as: Stage + + use GenStage + + @enforce_keys [:conn, :state, :opts] + defstruct [:conn, :state, :opts] + + @start_opts [:name, :spawn_opt, :debug] + @stage_opts [:demand, :buffer_size, :buffer_keep, :dispatcher] + + @doc """ + Start link a `GenStage` producer that will prepare a query, execute it and + stream results using a cursor. + + ### Options + + * `:pool_timeout` - The maximum time to wait for a reply when making a + synchronous call to the pool (default: `5_000`) + * `:queue` - Whether to block waiting in an internal queue for the + connection's state (boolean, default: `true`) + * `:timeout` - The maximum time that the caller is allowed the + to hold the connection's state (ignored when using a run/transaction + connection, default: `15_000`) + * `:log` - A function to log information about a call, either + a 1-arity fun, `{module, function, args}` with `DBConnection.LogEntry.t` + prepended to `args` or `nil`. See `DBConnection.LogEntry` (default: `nil`) + * `:flat_map` - A function to flat map stream results, either a 2-arity fun + or `{module, function, args}` with `conn` and `result` prepended to `args`, + or `nil`. + + The pool and connection module may support other options. All options + are passed to `handle_begin/2`, `handle_prepare/3, `handle_close/3, + `handle_declare/4`, `handle_first/4`, `handle_next/4`, `handle_deallocate/4`, + `handle_commit/2` and `handle_rollback/2`. + + ### Example + + query = %Query{statement: "SELECT id FROM table"} + {:ok, stage} = DBConnection.StreamStage.prepare_stream_link(conn, query, []) + stage |> Flow.from_stage() |> Enum.to_list() + end) + """ + def prepare_stream_link(pool, query, params, opts \\ []) do + start_link(pool, &DBConnection.prepare_stream/4, query, params, opts) + end + + @doc """ + Start link a `GenStage` producer that will execute a query and stream results + using a cursor. + + ### Options + + * `:pool_timeout` - The maximum time to wait for a reply when making a + synchronous call to the pool (default: `5_000`) + * `:queue` - Whether to block waiting in an internal queue for the + connection's state (boolean, default: `true`) + * `:timeout` - The maximum time that the caller is allowed the + to hold the connection's state (ignored when using a run/transaction + connection, default: `15_000`) + * `:log` - A function to log information about a call, either + a 1-arity fun, `{module, function, args}` with `DBConnection.LogEntry.t` + prepended to `args` or `nil`. See `DBConnection.LogEntry` (default: `nil`) + * `:flat_map` - A function to flat map stream results, either a 2-arity fun + or `{module, function, args}` with `conn` and `result` prepended to `args`, + or `nil`. + + The pool and connection module may support other options. All options + are passed to `handle_begin/2`, `handle_declare/4`, `handle_first/4`, + `handle_next/4`, `handle_deallocate/4`, `handle_commit/2` and + `handle_rollback/2`. + + ### Example + + query = %Query{statement: "SELECT id FROM table"} + {:ok, stage} = DBConnection.StreamStage.stream_link(conn, query, []) + stage |> Flow.from_stage() |> Enum.to_list() + end) + """ + def stream_link(pool, query, params, opts \\ []) do + start_link(pool, &DBConnection.stream/4, query, params, opts) + end + + @doc false + def init({pool, stream, query, params, opts}) do + {stage_opts, db_opts} = Keyword.split(opts, @stage_opts) + conn = DBConnection.stage_begin(pool, db_opts) + init = &start(&1, stream, query, params, db_opts) + case DBConnection.stage_transaction(conn, init, db_opts) do + {:ok, state} -> + {:producer, %Stage{conn: conn, state: state, opts: db_opts}, stage_opts} + {:error, reason} -> + exit(reason) + end + end + + @doc false + def handle_demand(demand, stage) do + %Stage{conn: conn, state: state, opts: opts} = stage + fun = fn(_) -> next(state, demand) end + case DBConnection.stage_transaction(conn, fun, opts) do + {:ok, {events, state}} -> + {:noreply, events, %Stage{stage | state: state}} + {:error, reason} -> + exit(reason) + :closed -> + raise DBConnection.ConnectionError, "connection is closed" + end + end + + @doc false + def terminate(reason, %Stage{conn: conn, state: state, opts: opts}) do + case DBConnection.stage_transaction(conn, fn(_) -> stop(state) end, opts) do + {:ok, _} when reason == :normal -> + DBConnection.stage_commit(conn, opts) + {:ok, _} -> + DBConnection.stage_rollback(conn, opts) + {:error, reason} -> + exit(reason) + :closed -> + :ok + end + end + + ## Helpers + + defp start_link(pool, stream_fun, query, params, opts) do + {start_opts, opts} = Keyword.split(opts, @start_opts) + args = {pool, stream_fun, query, params, opts} + GenStage.start_link(__MODULE__, args, start_opts) + end + + defp start(conn, stream_fun, query, params, opts) do + stream = stream_fun.(conn, query, params, opts) + {:suspended, _, cont} = start(conn, stream, opts) + {:cont, cont} + end + + defp start(conn, stream, opts) do + stream + |> flat_map(conn, opts) + |> Enumerable.reduce({:suspend, {0, []}}, &stream_reduce/2) + end + + defp flat_map(stream, conn, opts) do + case Keyword.get(opts, :flat_map) do + nil -> + stream + map when is_function(map, 2) -> + Stream.flat_map(stream, fn(elem) -> map.(conn, elem) end) + {mod, fun, args} -> + map = fn(elem) -> apply(mod, fun, [conn, elem | args]) end + Stream.flat_map(stream, map) + end + end + + defp stream_reduce(v, {1, acc}) do + {:suspend, {0, [v | acc]}} + end + defp stream_reduce(v, {n, acc}) do + {:cont, {n-1, [v | acc]}} + end + + defp next({:cont, cont}, n) when n > 0 do + case cont.({:cont, {n, []}}) do + {:suspended, {0, acc}, cont} -> + {Enum.reverse(acc), {:cont, cont}} + {state, {_, acc}} when state in [:halted, :done] -> + GenStage.async_notify(self(), {:producer, state}) + {Enum.reverse(acc), state} + end + end + defp next(state, _) when state in [:halted, :done] do + GenStage.async_notify(self(), {:producer, state}) + {[], state} + end + + defp stop({:cont, cont}) do + _ = cont.({:halt, {0, []}}) + :ok + end + defp stop(state) when state in [:halted, :done] do + :ok + end +end diff --git a/mix.exs b/mix.exs index 8844a7e1..307994e6 100644 --- a/mix.exs +++ b/mix.exs @@ -30,6 +30,8 @@ defmodule DBConnection.Mixfile do [{:connection, "~> 1.0.2"}, {:poolboy, "~> 1.5", [optional: true]}, {:sbroker, "~> 1.0", [optional: true]}, + {:gen_stage, "~> 0.11", [optional: true]}, + {:flow, "~> 0.11", [optional: true]}, {:ex_doc, "~> 0.12", only: :dev}] end diff --git a/mix.lock b/mix.lock index 725242b0..1c93c1e7 100644 --- a/mix.lock +++ b/mix.lock @@ -1,5 +1,7 @@ %{"connection": {:hex, :connection, "1.0.4", "a1cae72211f0eef17705aaededacac3eb30e6625b04a6117c1b2db6ace7d5976", [:mix], []}, "earmark": {:hex, :earmark, "0.2.1", "ba6d26ceb16106d069b289df66751734802777a3cbb6787026dd800ffeb850f3", [:mix], []}, "ex_doc": {:hex, :ex_doc, "0.12.0", "b774aabfede4af31c0301aece12371cbd25995a21bb3d71d66f5c2fe074c603f", [:mix], [{:earmark, "~> 0.2", [hex: :earmark, optional: false]}]}, + "flow": {:hex, :flow, "0.11.1", "cbc35a0236520cc5fec7b5863cd8431cb1e77297c5c9119055676355eb1fb5a6", [:mix], [{:gen_stage, "~> 0.11.0", [hex: :gen_stage, optional: false]}]}, + "gen_stage": {:hex, :gen_stage, "0.11.0", "943bdfa85c75fa624e0a36a9d135baad20a523be040178f5a215444b45c66ea4", [:mix], []}, "poolboy": {:hex, :poolboy, "1.5.1", "6b46163901cfd0a1b43d692657ed9d7e599853b3b21b95ae5ae0a777cf9b6ca8", [:rebar], []}, "sbroker": {:hex, :sbroker, "1.0.0", "28ff1b5e58887c5098539f236307b36fe1d3edaa2acff9d6a3d17c2dcafebbd0", [:rebar3], []}} diff --git a/test/test_support.exs b/test/test_support.exs index cce1cd73..c6138b5b 100644 --- a/test/test_support.exs +++ b/test/test_support.exs @@ -70,8 +70,6 @@ defmodule TestConnection do def start_link(opts), do: DBConnection.start_link(__MODULE__, opts) def connect(opts) do - agent = Keyword.fetch!(opts, :agent) - _ = Process.put(:agent, agent) TestAgent.eval(:connect, [opts]) end @@ -136,7 +134,6 @@ defmodule TestConnection do end end - defmodule TestQuery do defstruct [:state] end @@ -179,7 +176,11 @@ defmodule TestAgent do ok end - def eval(agent \\ Process.get(:agent), fun, args) do + def eval(fun, args) do + eval(get_agent(args), fun, args) + end + + def eval(agent, fun, args) do action = {fun, args} case Agent.get_and_update(agent, &get_and_update(&1, action)) do fun when is_function(fun) -> @@ -189,6 +190,23 @@ defmodule TestAgent do end end + defp get_agent(args) do + case Process.get(:agent) do + agent when is_pid(agent) -> + agent + nil -> + opts = get_opts(args) + agent = Keyword.fetch!(opts, :agent) + _ = Process.put(:agent, agent) + agent + end + end + + defp get_opts([opts]), do: opts + defp get_opts([opts, _]), do: opts + defp get_opts([_, opts, _]), do: opts + defp get_opts([_, _, opts, _]), do: opts + def record(agent) do Enum.reverse(Agent.get(agent, &elem(&1, 1))) end From 6bf13646c64c64838ec995774002d411f0a6cc30 Mon Sep 17 00:00:00 2001 From: James Fish Date: Mon, 10 Apr 2017 01:49:48 +0100 Subject: [PATCH 02/20] Move DBConnection.StreamStage to generic DBConnection.Stage --- .../{stream_stage_test.exs => stage_test.exs} | 41 +- integration_test/tests.exs | 2 +- lib/db_connection/stage.ex | 349 ++++++++++++++++++ lib/db_connection/stream_stage.ex | 189 ---------- test/test_support.exs | 8 + 5 files changed, 364 insertions(+), 225 deletions(-) rename integration_test/cases/{stream_stage_test.exs => stage_test.exs} (62%) create mode 100644 lib/db_connection/stage.ex delete mode 100644 lib/db_connection/stream_stage.ex diff --git a/integration_test/cases/stream_stage_test.exs b/integration_test/cases/stage_test.exs similarity index 62% rename from integration_test/cases/stream_stage_test.exs rename to integration_test/cases/stage_test.exs index e2beabbc..21e9dbff 100644 --- a/integration_test/cases/stream_stage_test.exs +++ b/integration_test/cases/stage_test.exs @@ -2,7 +2,6 @@ defmodule StreamStageTest do use ExUnit.Case, async: true alias TestPool, as: P - alias DBConnection.StreamStage, as: S alias TestAgent, as: A alias TestQuery, as: Q alias TestCursor, as: C @@ -22,10 +21,11 @@ defmodule StreamStageTest do opts = [agent: agent, parent: self()] {:ok, pool} = P.start_link(opts) - {:ok, stage} = S.stream_link(pool, %Q{}, [:param], opts) + {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) + mon = Process.monitor(stage) assert stage |> Flow.from_stage() |> Enum.to_list() == [%R{}, %R{}] - GenStage.stop(stage) + assert_receive {:DOWN, ^mon, :process, ^stage, :normal} assert [ connect: [_], @@ -53,10 +53,11 @@ defmodule StreamStageTest do opts = [agent: agent, parent: self()] {:ok, pool} = P.start_link(opts) - {:ok, stage} = S.prepare_stream_link(pool, %Q{}, [:param], opts) + {:ok, stage} = P.prepare_stream_stage(pool, %Q{}, [:param], opts) + mon = Process.monitor(stage) assert stage |> Flow.from_stage() |> Enum.to_list() == [%R{}, %R{}] - GenStage.stop(stage) + assert_receive {:DOWN, ^mon, :process, ^stage, :normal} assert [ connect: [_], @@ -69,34 +70,4 @@ defmodule StreamStageTest do handle_commit: [_, :newer_state2] ] = A.record(agent) end - - test "stream can be rolled back in flat map" do - stack = [ - {:ok, :state}, - {:ok, :began, :new_state}, - {:ok, %C{}, :newer_state}, - {:ok, %R{}, :newest_state}, - {:ok, :deallocated, :state2}, - {:ok, :rolledback, :new_state2} - ] - {:ok, agent} = A.start_link(stack) - - opts = [agent: agent, parent: self()] - {:ok, pool} = P.start_link(opts) - map = fn(conn, _) -> P.rollback(conn, :normal) end - {:ok, stage} = S.stream_link(pool, %Q{}, [:param], [flat_map: map] ++ opts) - mon = Process.monitor(stage) - catch_exit(stage |> Flow.from_stage() |> Enum.to_list()) - - assert_receive {:DOWN, ^ref, :process, ^stage, :normal} - - assert [ - connect: [_], - handle_begin: [_, :state], - handle_declare: [%Q{}, [:param], _, :new_state], - handle_first: [%Q{}, %C{}, _, :newer_state], - handle_deallocate: [%Q{}, %C{}, _, :newest_state], - handle_rollback: [_, :state2] - ] = A.record(agent) - end end diff --git a/integration_test/tests.exs b/integration_test/tests.exs index ada91813..da99142e 100644 --- a/integration_test/tests.exs +++ b/integration_test/tests.exs @@ -9,7 +9,7 @@ Code.require_file "cases/prepare_execute_test.exs", __DIR__ Code.require_file "cases/prepare_stream_test.exs", __DIR__ Code.require_file "cases/prepare_test.exs", __DIR__ Code.require_file "cases/queue_test.exs", __DIR__ -Code.require_file "cases/stream_stage_test.exs", __DIR__ +Code.require_file "cases/stage_test.exs", __DIR__ Code.require_file "cases/stream_test.exs", __DIR__ Code.require_file "cases/transaction_execute_test.exs", __DIR__ Code.require_file "cases/transaction_test.exs", __DIR__ diff --git a/lib/db_connection/stage.ex b/lib/db_connection/stage.ex new file mode 100644 index 00000000..d9273a6f --- /dev/null +++ b/lib/db_connection/stage.ex @@ -0,0 +1,349 @@ +defmodule DBConnection.Stage do + @moduledoc """ + A `GenStage` process that encapsulates a transaction. + """ + alias __MODULE__, as: Stage + + use GenStage + + @enforce_keys [:conn, :handle, :stop, :state, :opts, :type] + defstruct [:conn, :handle, :stop, :state, :opts, :type, + consumers: [], producers: %{}, active: [], done?: false] + + @start_opts [:name, :spawn_opt, :debug] + @stage_opts [:demand, :buffer_size, :buffer_keep, :dispatcher, :subscribe_to] + + @doc """ + Start link a `GenStage` producer that will prepare a query, execute it and + stream results using a cursor inside a transaction. + + The transaction is rolled back if the process terminates with a reason other + than `:normal`. + + ### Options + + * `:pool_timeout` - The maximum time to wait for a reply when making a + synchronous call to the pool (default: `5_000`) + * `:queue` - Whether to block waiting in an internal queue for the + connection's state (boolean, default: `true`) + * `:timeout` - The maximum time that the caller is allowed the + to hold the connection's state (ignored when using a run/transaction + connection, default: `15_000`) + * `:log` - A function to log information about a call, either + a 1-arity fun, `{module, function, args}` with `DBConnection.LogEntry.t` + prepended to `args` or `nil`. See `DBConnection.LogEntry` (default: `nil`) + + The pool and connection module may support other options. All options + are passed to `handle_begin/2`, `handle_prepare/3, `handle_close/3, + `handle_declare/4`, `handle_first/4`, `handle_next/4`, `handle_deallocate/4`, + `handle_commit/2` and `handle_rollback/2`. + + ### Example + + query = %Query{statement: "SELECT id FROM table"} + {:ok, stage} = DBConnection.StreamStage.prepare_stream_link(conn, query, []) + stage |> Flow.from_stage() |> Enum.to_list() + end) + """ + def prepare_stream(pool, query, params, opts \\ []) do + stream(pool, &DBConnection.prepare_stream/4, query, params, opts) + end + + @doc """ + Start link a `GenStage` producer that will execute a query and stream results + using a cursor inside a transaction. + + The transaction is rolled back if the process terminates with a reason other + than `:normal`. + + ### Options + + * `:pool_timeout` - The maximum time to wait for a reply when making a + synchronous call to the pool (default: `5_000`) + * `:queue` - Whether to block waiting in an internal queue for the + connection's state (boolean, default: `true`) + * `:timeout` - The maximum time that the caller is allowed the + to hold the connection's state (ignored when using a run/transaction + connection, default: `15_000`) + * `:log` - A function to log information about a call, either + a 1-arity fun, `{module, function, args}` with `DBConnection.LogEntry.t` + prepended to `args` or `nil`. See `DBConnection.LogEntry` (default: `nil`) + + The pool and connection module may support other options. All options + are passed to `handle_begin/2`, `handle_declare/4`, `handle_first/4`, + `handle_next/4`, `handle_deallocate/4`, `handle_commit/2` and + `handle_rollback/2`. + + ### Example + + query = %Query{statement: "SELECT id FROM table"} + {:ok, stage} = DBConnection.StreamStage.stream_link(conn, query, []) + stage |> Flow.from_stage() |> Enum.to_list() + end) + """ + def stream(pool, query, params, opts \\ []) do + stream(pool, &DBConnection.stream/4, query, params, opts) + end + + @doc """ + Start link a `GenStage` process that will run a transaction for its duration. + + The first argument is the pool, the second argument is the `GenStage` type, + the third argument is the start function, the fourth argument is the handle + function, the fifth argument is the stop function and the optional sixth + argument are the options. + + The start function is a 1-arity anonymous function with argument + `DBConnection.t`. This is called after the transaction begins but before + `start_link/6` returns. It should return the `state` or call + `DBConnection.rollback/2` to stop the `GenStage`. + + The handle function is a 3-arity anonymous function. The first argument is the + `DBConnection.t` for the transaction and the third argument is the state. + If the `GenStage` type is a `:producer`, then the second argument is the + `demand` from a `GenStage` `handle_demand` callback. Otherwise the second + argument is the events from a `GenStage` `handle_events` callback. This + function returns a 2-tuple, with first element as events (empty list for + `:consumer`) and second element as the `state`. This function can roll back + and stop the `GenStage` using `DBConnection.rollback/2`. + + The stop function is a 3-arity anonymous function. The first argument is the + `DBConnection.t` for the transaction, the second argument is the terminate + reason and the third argument is the `state`. This function will only be + called if connection is alive and the transaction has not been rolled back. If + this function returns the transaction is commited. This function can roll back + and stop the `GenStage` using `DBConnection.rollback/2`. + + The `GenStage` process will behave like a `Flow` stage: + + * It will stop with reason `:normal` when the last consumer cancels + * It will notify consumers that it is done when all producers have cancelled + or notified that they are done or halted + * It will cancel all remaining producers when all producers have notified + that they are done or halted + + ### Options + + * `:name` - A name to register the started process (see the `:name` option + in `GenServer.start_link/3`) + * `:pool_timeout` - The maximum time to wait for a reply when making a + synchronous call to the pool (default: `5_000`) + * `:queue` - Whether to block waiting in an internal queue for the + connection's state (boolean, default: `true`) + * `:timeout` - The maximum time that the caller is allowed the + to hold the connection's state (ignored when using a run/transaction + connection, default: `15_000`) + * `:log` - A function to log information about a call, either + a 1-arity fun, `{module, function, args}` with `DBConnection.LogEntry.t` + prepended to `args` or `nil`. See `DBConnection.LogEntry` (default: `nil`) + + The pool and connection module may support other options. All options + are passed to `handle_begin/2`, `handle_commit/2` and `handle_rollback/2`. + All options are passed to the `GenStage` on init. + + ### Example + + start = &DBConnection.prepare!(&1, %2, opts) + handle = + fn(conn, param, query) -> + {[DBConection.execute!(conn, param, query, opts)], query} + end + stop = %DBConnection.close(&1, &2, opts) + DBConnection.Stage.start_link(pool, :producer_consumer, + start, handle, stop, opts) + """ + @spec start_link(GenServer.server, :producer, + ((DBConnection.t) -> state), + ((DBConnection.t, demand :: pos_integer, state) -> {[any], state}), + ((DBConnection.t, reason :: any, state) -> any), Keyword.t) :: + GenServer.on_start when state: var + @spec start_link(GenServer.server, :producer_consumer, + ((DBConnection.t) -> state), + ((DBConnection.t, [any], state) -> {[any], state}), + ((DBConnection.t, reason :: any, state) -> any), Keyword.t) :: + GenServer.on_start when state: var + @spec start_link(GenServer.server, :consumer, + ((DBConnection.t) -> state), + ((DBConnection.t, [any], state) -> {[], state}), + ((DBConnection.t, reason :: any, state) -> any), Keyword.t) :: + GenServer.on_start when state: var + def start_link(pool, type, start, handle, stop, opts \\ []) do + start_opts = Keyword.take(opts, @start_opts) + args = {pool, type, start, handle, stop, opts} + GenStage.start_link(__MODULE__, args, start_opts) + end + + @doc false + def init({pool, type, start, handle, stop, opts}) do + stage_opts = Keyword.take(opts, @stage_opts) + conn = DBConnection.stage_begin(pool, opts) + case DBConnection.stage_transaction(conn, start, opts) do + {:ok, state} -> + stage = %Stage{conn: conn, handle: handle, stop: stop, state: state, + opts: opts, type: type} + {type, stage, stage_opts} + {:error, reason} -> + exit(reason) + end + end + + @doc false + def handle_subscribe(:producer, _, {pid, ref}, stage) do + %Stage{producers: producers, active: active} = stage + stage = %Stage{stage | producers: Map.put(producers, ref, pid), + active: [ref | active]} + {:automatic, stage} + end + def handle_subscribe(:consumer, _, {_, ref}, stage) do + %Stage{consumers: consumers} = stage + {:automatic, %Stage{stage | consumers: [ref | consumers]}} + end + + @doc false + def handle_cancel(_, {_, ref}, stage) do + %Stage{type: type, consumers: consumers, producers: producers, + active: active, done?: done?} = stage + case producers do + %{^ref => _} when active != [ref] or done? -> + producers = Map.delete(producers, ref) + active = List.delete(active, ref) + {:noreply, [], %Stage{stage | active: active, producers: producers}} + %{^ref => _} when type == :consumer -> + producers = Map.delete(producers, ref) + for {ref, pid} <- producers do + GenStage.cancel({pid, ref}, :normal, [:noconnect]) + end + stage = %Stage{stage | active: [], done?: true, producers: producers} + {:noreply, [], stage} + %{^ref => _} when type == :producer_consumer -> + producers = Map.delete(producers, ref) + GenStage.async_notify(self(), {:producer, :done}) + stage = %Stage{stage | active: [], done?: true, producers: producers} + {:noreply, [], stage} + %{} when consumers == [ref] -> + {:stop, :normal, %Stage{stage | consumers: []}} + %{} -> + consumers = List.delete(consumers, ref) + {:noreply, [], %Stage{stage | consumers: consumers}} + end + end + + def handle_info({{_, ref}, {:producer, state}}, stage) when state in [:halted, :done] do + %Stage{type: type, producers: producers, active: active, + done?: done?} = stage + case producers do + %{^ref => _} when active != [ref] or done? -> + active = List.delete(active, ref) + {:noreply, [], %Stage{stage | active: active}} + %{^ref => _} when type == :consumer -> + for {ref, pid} <- producers do + GenStage.cancel({pid, ref}, :normal, [:noconnect]) + end + {:noreply, [], %Stage{stage | active: [], done?: true}} + %{^ref => _} when type == :producer_consumer -> + GenStage.async_notify(self(), {:producer, :done}) + {:noreply, [], %Stage{stage | active: [], done?: true}} + %{} -> + {:noreply, [], stage} + end + end + def handle_info(_, stage) do + {:noreply, [], stage} + end + + @doc false + def handle_demand(demand, stage) do + %Stage{conn: conn, handle: handle, state: state, opts: opts} = stage + fun = &handle.(&1, demand, state) + case DBConnection.stage_transaction(conn, fun, opts) do + {:ok, {events, state}} -> + {:noreply, events, %Stage{stage | state: state}} + {:error, reason} -> + exit(reason) + :closed -> + raise DBConnection.ConnectionError, "connection is closed" + end + end + + @doc false + def handle_events(events, _, stage) do + %Stage{conn: conn, handle: handle, state: state, opts: opts} = stage + fun = &handle.(&1, events, state) + case DBConnection.stage_transaction(conn, fun, opts) do + {:ok, {events, state}} -> + {:noreply, events, %Stage{stage | state: state}} + {:error, reason} -> + exit(reason) + :closed -> + raise DBConnection.ConnectionError, "connection is closed" + end + end + + @doc false + def terminate(reason, stage) do + %Stage{conn: conn, stop: stop, state: state, opts: opts} = stage + fun = &stop.(&1, reason, state) + case DBConnection.stage_transaction(conn, fun, opts) do + {:ok, _} -> + case DBConnection.stage_commit(conn, opts) do + :ok -> + :ok + {:error, :rollback} -> + exit(:rollback) + end + {:error, new_reason} -> + DBConnection.stage_rollback(conn, opts) + if new_reason != reason, do: exit(reason) + :closed -> + :ok + end + end + + ## Helpers + + defp stream(pool, stream_fun, query, params, opts) do + start = &stream_start(&1, stream_fun, query, params, opts) + start_link(pool, :producer, start, &stream_next/3, &stream_stop/3, opts) + end + + defp stream_start(conn, stream_fun, query, params, opts) do + stream = stream_fun.(conn, query, params, opts) + {:suspended, _, cont} = Enumerable.reduce(stream, {:suspend, {0, []}}, &stream_reduce/2) + {:cont, cont} + end + + defp stream_reduce(v, {1, acc}) do + {:suspend, {0, [v | acc]}} + end + defp stream_reduce(v, {n, acc}) do + {:cont, {n-1, [v | acc]}} + end + + defp stream_next(_, n, {:cont, cont}) when n > 0 do + case cont.({:cont, {n, []}}) do + {:suspended, {0, acc}, cont} -> + {Enum.reverse(acc), {:cont, cont}} + {state, {_, acc}} when state in [:halted, :done] -> + GenStage.async_notify(self(), {:producer, state}) + {Enum.reverse(acc), state} + end + end + defp stream_next(_, _, state) when state in [:halted, :done] do + {[], state} + end + + defp stream_stop(conn, reason, {:cont, cont}) do + _ = cont.({:halt, {0, []}}) + stream_stop(conn, reason) + end + defp stream_stop(conn, reason, state) when state in [:halted, :done] do + stream_stop(conn, reason) + end + + defp stream_stop(_, :normal) do + :ok + end + defp stream_stop(conn, reason) do + DBConnection.rollback(conn, reason) + end +end diff --git a/lib/db_connection/stream_stage.ex b/lib/db_connection/stream_stage.ex deleted file mode 100644 index 9cbedcfa..00000000 --- a/lib/db_connection/stream_stage.ex +++ /dev/null @@ -1,189 +0,0 @@ -defmodule DBConnection.StreamStage do - @moduledoc """ - A `GenStage` producer that streams the results of a query inside a - transaction. - """ - alias __MODULE__, as: Stage - - use GenStage - - @enforce_keys [:conn, :state, :opts] - defstruct [:conn, :state, :opts] - - @start_opts [:name, :spawn_opt, :debug] - @stage_opts [:demand, :buffer_size, :buffer_keep, :dispatcher] - - @doc """ - Start link a `GenStage` producer that will prepare a query, execute it and - stream results using a cursor. - - ### Options - - * `:pool_timeout` - The maximum time to wait for a reply when making a - synchronous call to the pool (default: `5_000`) - * `:queue` - Whether to block waiting in an internal queue for the - connection's state (boolean, default: `true`) - * `:timeout` - The maximum time that the caller is allowed the - to hold the connection's state (ignored when using a run/transaction - connection, default: `15_000`) - * `:log` - A function to log information about a call, either - a 1-arity fun, `{module, function, args}` with `DBConnection.LogEntry.t` - prepended to `args` or `nil`. See `DBConnection.LogEntry` (default: `nil`) - * `:flat_map` - A function to flat map stream results, either a 2-arity fun - or `{module, function, args}` with `conn` and `result` prepended to `args`, - or `nil`. - - The pool and connection module may support other options. All options - are passed to `handle_begin/2`, `handle_prepare/3, `handle_close/3, - `handle_declare/4`, `handle_first/4`, `handle_next/4`, `handle_deallocate/4`, - `handle_commit/2` and `handle_rollback/2`. - - ### Example - - query = %Query{statement: "SELECT id FROM table"} - {:ok, stage} = DBConnection.StreamStage.prepare_stream_link(conn, query, []) - stage |> Flow.from_stage() |> Enum.to_list() - end) - """ - def prepare_stream_link(pool, query, params, opts \\ []) do - start_link(pool, &DBConnection.prepare_stream/4, query, params, opts) - end - - @doc """ - Start link a `GenStage` producer that will execute a query and stream results - using a cursor. - - ### Options - - * `:pool_timeout` - The maximum time to wait for a reply when making a - synchronous call to the pool (default: `5_000`) - * `:queue` - Whether to block waiting in an internal queue for the - connection's state (boolean, default: `true`) - * `:timeout` - The maximum time that the caller is allowed the - to hold the connection's state (ignored when using a run/transaction - connection, default: `15_000`) - * `:log` - A function to log information about a call, either - a 1-arity fun, `{module, function, args}` with `DBConnection.LogEntry.t` - prepended to `args` or `nil`. See `DBConnection.LogEntry` (default: `nil`) - * `:flat_map` - A function to flat map stream results, either a 2-arity fun - or `{module, function, args}` with `conn` and `result` prepended to `args`, - or `nil`. - - The pool and connection module may support other options. All options - are passed to `handle_begin/2`, `handle_declare/4`, `handle_first/4`, - `handle_next/4`, `handle_deallocate/4`, `handle_commit/2` and - `handle_rollback/2`. - - ### Example - - query = %Query{statement: "SELECT id FROM table"} - {:ok, stage} = DBConnection.StreamStage.stream_link(conn, query, []) - stage |> Flow.from_stage() |> Enum.to_list() - end) - """ - def stream_link(pool, query, params, opts \\ []) do - start_link(pool, &DBConnection.stream/4, query, params, opts) - end - - @doc false - def init({pool, stream, query, params, opts}) do - {stage_opts, db_opts} = Keyword.split(opts, @stage_opts) - conn = DBConnection.stage_begin(pool, db_opts) - init = &start(&1, stream, query, params, db_opts) - case DBConnection.stage_transaction(conn, init, db_opts) do - {:ok, state} -> - {:producer, %Stage{conn: conn, state: state, opts: db_opts}, stage_opts} - {:error, reason} -> - exit(reason) - end - end - - @doc false - def handle_demand(demand, stage) do - %Stage{conn: conn, state: state, opts: opts} = stage - fun = fn(_) -> next(state, demand) end - case DBConnection.stage_transaction(conn, fun, opts) do - {:ok, {events, state}} -> - {:noreply, events, %Stage{stage | state: state}} - {:error, reason} -> - exit(reason) - :closed -> - raise DBConnection.ConnectionError, "connection is closed" - end - end - - @doc false - def terminate(reason, %Stage{conn: conn, state: state, opts: opts}) do - case DBConnection.stage_transaction(conn, fn(_) -> stop(state) end, opts) do - {:ok, _} when reason == :normal -> - DBConnection.stage_commit(conn, opts) - {:ok, _} -> - DBConnection.stage_rollback(conn, opts) - {:error, reason} -> - exit(reason) - :closed -> - :ok - end - end - - ## Helpers - - defp start_link(pool, stream_fun, query, params, opts) do - {start_opts, opts} = Keyword.split(opts, @start_opts) - args = {pool, stream_fun, query, params, opts} - GenStage.start_link(__MODULE__, args, start_opts) - end - - defp start(conn, stream_fun, query, params, opts) do - stream = stream_fun.(conn, query, params, opts) - {:suspended, _, cont} = start(conn, stream, opts) - {:cont, cont} - end - - defp start(conn, stream, opts) do - stream - |> flat_map(conn, opts) - |> Enumerable.reduce({:suspend, {0, []}}, &stream_reduce/2) - end - - defp flat_map(stream, conn, opts) do - case Keyword.get(opts, :flat_map) do - nil -> - stream - map when is_function(map, 2) -> - Stream.flat_map(stream, fn(elem) -> map.(conn, elem) end) - {mod, fun, args} -> - map = fn(elem) -> apply(mod, fun, [conn, elem | args]) end - Stream.flat_map(stream, map) - end - end - - defp stream_reduce(v, {1, acc}) do - {:suspend, {0, [v | acc]}} - end - defp stream_reduce(v, {n, acc}) do - {:cont, {n-1, [v | acc]}} - end - - defp next({:cont, cont}, n) when n > 0 do - case cont.({:cont, {n, []}}) do - {:suspended, {0, acc}, cont} -> - {Enum.reverse(acc), {:cont, cont}} - {state, {_, acc}} when state in [:halted, :done] -> - GenStage.async_notify(self(), {:producer, state}) - {Enum.reverse(acc), state} - end - end - defp next(state, _) when state in [:halted, :done] do - GenStage.async_notify(self(), {:producer, state}) - {[], state} - end - - defp stop({:cont, cont}) do - _ = cont.({:halt, {0, []}}) - :ok - end - defp stop(state) when state in [:halted, :done] do - :ok - end -end diff --git a/test/test_support.exs b/test/test_support.exs index c6138b5b..74431acd 100644 --- a/test/test_support.exs +++ b/test/test_support.exs @@ -63,6 +63,14 @@ defmodule TestConnection do DBConnection.close!(pool, query, opts2 ++ unquote(opts)) end + def stream_stage(pool, query, params, opts2 \\ []) do + DBConnection.Stage.stream(pool, query, params, opts2 ++ unquote(opts)) + end + + def prepare_stream_stage(pool, query, params, opts2 \\ []) do + DBConnection.Stage.prepare_stream(pool, query, params, opts2 ++ unquote(opts)) + end + defoverridable [start_link: 1] end end From eb9482984a3b04663fcf85d2e96a58de18b52615 Mon Sep 17 00:00:00 2001 From: James Fish Date: Mon, 10 Apr 2017 19:59:22 +0100 Subject: [PATCH 03/20] Improve stage tests --- integration_test/cases/stage_test.exs | 649 +++++++++++++++++++++++++- lib/db_connection/stage.ex | 20 +- test/test_support.exs | 5 + 3 files changed, 668 insertions(+), 6 deletions(-) diff --git a/integration_test/cases/stage_test.exs b/integration_test/cases/stage_test.exs index 21e9dbff..7ab27f4f 100644 --- a/integration_test/cases/stage_test.exs +++ b/integration_test/cases/stage_test.exs @@ -1,4 +1,4 @@ -defmodule StreamStageTest do +defmodule StageTest do use ExUnit.Case, async: true alias TestPool, as: P @@ -7,6 +7,352 @@ defmodule StreamStageTest do alias TestCursor, as: C alias TestResult, as: R + test "producer executes on demand" do + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, %Q{}, :newer_state}, + {:ok, %R{}, :newest_state}, + {:ok, :closed, :state2}, + {:ok, :commited, :new_state2} + ] + {:ok, agent} = A.start_link(stack) + + opts = [agent: agent, parent: self()] + {:ok, pool} = P.start_link(opts) + start = &DBConnection.prepare!(&1, %Q{}) + handle = fn(conn, demand, query) -> + GenStage.async_notify(self(), {:producer, :done}) + {[DBConnection.execute!(conn, query, [demand])], query} + end + stop = fn(conn, _, query) -> DBConnection.close!(conn, query) end + {:ok, stage} = P.stage(pool, :producer, start, handle, stop, opts) + mon = Process.monitor(stage) + assert stage |> Flow.from_stage() |> Enum.to_list() == [%R{}] + + assert_receive {:DOWN, ^mon, :process, ^stage, :normal} + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_prepare: [%Q{}, _, :new_state], + handle_execute: [%Q{}, _, _, :newer_state], + handle_close: [%Q{}, _, :newest_state], + handle_commit: [_, :state2] + ] = A.record(agent) + end + + test "producer exits when last consumer cancels" do + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, %Q{}, :newer_state}, + {:ok, %R{}, :newest_state}, + {:ok, :closed, :state2}, + {:ok, :commited, :new_state2} + ] + {:ok, agent} = A.start_link(stack) + + parent = self() + opts = [agent: agent, parent: parent] + {:ok, pool} = P.start_link(opts) + start = &DBConnection.prepare!(&1, %Q{}) + handle = fn(conn, demand, query) -> + GenStage.async_notify(self(), {:producer, :done}) + {[DBConnection.execute!(conn, query, [demand])], query} + end + stop = fn(conn, _, query) -> DBConnection.close!(conn, query) end + {:ok, stage} = P.stage(pool, :producer, start, handle, stop, opts) + + ref = make_ref() + sub_opts = [cancel: :temporary] + send(stage, {:"$gen_producer", {parent, ref}, {:subscribe, nil, sub_opts}}) + sub = {stage, ref} + GenStage.ask(sub, 1) + assert_receive {:"$gen_consumer", ^sub, [%R{}]} + mon = Process.monitor(stage) + GenStage.cancel(sub, :normal) + assert_receive {:DOWN, ^mon, :process, ^stage, :normal} + assert_received {:"$gen_consumer", ^sub, {:cancel, :normal}} + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_prepare: [%Q{}, _, :new_state], + handle_execute: [%Q{}, [1], _, :newer_state], + handle_close: [%Q{}, _, :newest_state], + handle_commit: [_, :state2] + ] = A.record(agent) + end + + test "producer does not stop or commit on rollback" do + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, %Q{}, :newer_state}, + {:ok, :rolledback, :newest_state} + ] + {:ok, agent} = A.start_link(stack) + + opts = [agent: agent, parent: self()] + {:ok, pool} = P.start_link(opts) + start = &DBConnection.prepare!(&1, %Q{}) + handle = fn(conn, _, _) -> + DBConnection.rollback(conn, :normal) + end + stop = fn(conn, _, query) -> DBConnection.close!(conn, query) end + {:ok, stage} = P.stage(pool, :producer, start, handle, stop, opts) + mon = Process.monitor(stage) + catch_exit(stage |> Flow.from_stage() |> Enum.to_list()) + + assert_receive {:DOWN, ^mon, :process, ^stage, :normal} + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_prepare: [%Q{}, _, :new_state], + handle_rollback: [_, :newer_state] + ] = A.record(agent) + end + + test "producer does not stop or commit on nested rollback" do + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, %Q{}, :newer_state}, + {:ok, :rolledback, :newest_state} + ] + {:ok, agent} = A.start_link(stack) + + opts = [agent: agent, parent: self()] + {:ok, pool} = P.start_link(opts) + start = &DBConnection.prepare!(&1, %Q{}) + handle = fn(conn, _, query) -> + assert DBConnection.transaction(conn, fn(conn2) -> + DBConnection.rollback(conn2, :oops) + end) == {:error, :oops} + {[:oops], query} + end + stop = fn(conn, _, query) -> DBConnection.close!(conn, query) end + {:ok, stage} = P.stage(pool, :producer, start, handle, stop, opts) + mon = Process.monitor(stage) + assert stage |> Flow.from_stage() |> Enum.take(1) == [:oops] + + assert_receive {:DOWN, ^mon, :process, ^stage, :normal} + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_prepare: [%Q{}, _, :new_state], + handle_rollback: [_, :newer_state] + ] = A.record(agent) + end + + test "producer exits but does not stop or commit after nested rollback" do + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, :rolledback, :newer_state} + ] + {:ok, agent} = A.start_link(stack) + + opts = [agent: agent, parent: self()] + {:ok, pool} = P.start_link(opts) + start = fn(conn) -> + assert DBConnection.transaction(conn, fn(conn2) -> + DBConnection.rollback(conn2, :oops) + end) == {:error, :oops} + :oops + end + fail = fn(_, _, _) -> flunk "should not run" end + {:ok, stage} = P.stage(pool, :producer, start, fail, fail, opts) + + Process.flag(:trap_exit, true) + catch_exit(stage |> Flow.from_stage() |> Enum.to_list()) + assert_receive {:EXIT, ^stage, :rollback} + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_rollback: [_, :new_state] + ] = A.record(agent) + end + + test "consumer executes on events" do + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, %Q{}, :newer_state}, + {:ok, %R{}, :newest_state}, + {:ok, :closed, :state2}, + {:ok, :commited, :new_state2} + ] + {:ok, agent} = A.start_link(stack) + + opts = [agent: agent, parent: self()] + {:ok, pool} = P.start_link(opts) + start = &DBConnection.prepare!(&1, %Q{}) + handle = fn(conn, events, query) -> + DBConnection.execute!(conn, query, events) + {[], query} + end + stop = fn(conn, _, query) -> DBConnection.close!(conn, query) end + {:ok, stage} = P.stage(pool, :consumer, start, handle, stop, opts) + mon = Process.monitor(stage) + flow = Flow.from_enumerable([:param]) + {:ok, _} = Flow.into_stages(flow, [stage]) + + assert_receive {:DOWN, ^mon, :process, ^stage, :normal} + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_prepare: [%Q{}, _, :new_state], + handle_execute: [%Q{}, [:param], _, :newer_state], + handle_close: [%Q{}, _, :newest_state], + handle_commit: [_, :state2] + ] = A.record(agent) + end + + test "consumer cancels new producers after it's done" do + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, %Q{}, :newer_state}, + {:ok, %R{}, :newest_state}, + {:ok, :closed, :state2}, + {:ok, :commited, :new_state2} + ] + {:ok, agent} = A.start_link(stack) + + parent = self() + opts = [agent: agent, parent: parent] + {:ok, pool} = P.start_link(opts) + start = &DBConnection.prepare!(&1, %Q{}) + handle = fn(conn, events, query) -> + DBConnection.execute!(conn, query, events) + {[], query} + end + stop = fn(conn, _, query) -> DBConnection.close!(conn, query) end + {:ok, stage} = P.stage(pool, :consumer, start, handle, stop, opts) + + :ok = GenStage.async_subscribe(stage, [to: self(), cancel: :temporary, max_demand: 1]) + + assert_receive {:"$gen_producer", {^stage, ref1} = sub1, {:subscribe, _, _}} + assert_receive {:"$gen_producer", ^sub1, {:ask, 1}} + + send(stage, {:"$gen_consumer", {self(), ref1}, [:param]}) + send(stage, {{self(), ref1}, {:producer, :done}}) + :ok = GenStage.async_subscribe(stage, [to: self(), cancel: :temporary, max_demand: 1]) + + assert_receive {:"$gen_producer", {^stage, _} = sub2, {:subscribe, _, _}} + assert_receive {:"$gen_producer", ^sub1, {:ask, 1}} + assert_receive {:"$gen_producer", ^sub1, {:cancel, :normal}} + assert_receive {:"$gen_producer", ^sub2, {:cancel, :normal}} + + GenStage.stop(stage) + refute_received {:"$gen_producer", _, _} + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_prepare: [%Q{}, _, :new_state], + handle_execute: [%Q{}, [:param], _, :newer_state], + handle_close: [%Q{}, _, :newest_state], + handle_commit: [_, :state2] + ] = A.record(agent) + end + + test "producer_consumer executes on events" do + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, %Q{}, :newer_state}, + {:ok, %R{}, :newest_state}, + {:ok, :closed, :state2}, + {:ok, :commited, :new_state2} + ] + {:ok, agent} = A.start_link(stack) + + opts = [agent: agent, parent: self()] + {:ok, pool} = P.start_link(opts) + start = &DBConnection.prepare!(&1, %Q{}) + handle = fn(conn, events, query) -> + {[DBConnection.execute!(conn, query, events)], query} + end + stop = fn(conn, _, query) -> DBConnection.close!(conn, query) end + {:ok, stage} = P.stage(pool, :producer_consumer, start, handle, stop, opts) + mon = Process.monitor(stage) + {:ok, _} = Flow.from_enumerable([:param]) |> Flow.into_stages([stage]) + assert Flow.from_stage(stage) |> Enum.to_list() == [%R{}] + + assert_receive {:DOWN, ^mon, :process, ^stage, :normal} + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_prepare: [%Q{}, _, :new_state], + handle_execute: [%Q{}, [:param], _, :newer_state], + handle_close: [%Q{}, _, :newest_state], + handle_commit: [_, :state2] + ] = A.record(agent) + end + + test "producer_consumer does not send demand to new producers after done" do + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, %Q{}, :newer_state}, + {:ok, %R{}, :newest_state}, + {:ok, :closed, :state2}, + {:ok, :commited, :new_state2} + ] + {:ok, agent} = A.start_link(stack) + + parent = self() + opts = [agent: agent, parent: parent] + {:ok, pool} = P.start_link(opts) + start = &DBConnection.prepare!(&1, %Q{}) + handle = fn(conn, events, query) -> + {[DBConnection.execute!(conn, query, events)], query} + end + stop = fn(conn, _, query) -> DBConnection.close!(conn, query) end + {:ok, stage} = P.stage(pool, :producer_consumer, start, handle, stop, opts) + + :ok = GenStage.async_subscribe(stage, [to: self(), cancel: :temporary, max_demand: 1]) + + assert_receive {:"$gen_producer", {^stage, ref1} = sub1, {:subscribe, _, _}} + assert_receive {:"$gen_producer", ^sub1, {:ask, 1}} + + task = Task.async(fn() -> + [stage] |> Flow.from_stages() |> Enum.map(&send(parent, &1)) + end) + + send(stage, {:"$gen_consumer", {self(), ref1}, [:param]}) + assert_receive %R{} + + :sys.suspend(stage) + send(stage, {{self(), ref1}, {:producer, :done}}) + :ok = GenStage.async_subscribe(stage, [to: self(), cancel: :temporary, max_demand: 1]) + mon = Process.monitor(stage) + :sys.resume(stage) + + assert Task.await(task) == [%R{}] + assert_receive {:"$gen_producer", {^stage, _}, {:subscribe, _, _}} + assert_receive {:DOWN, ^mon, :process, ^stage, :normal} + assert_received {:"$gen_producer", ^sub1, {:ask, 1}} + refute_received {:"$gen_producer", _, _} + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_prepare: [%Q{}, _, :new_state], + handle_execute: [%Q{}, [:param], _, :newer_state], + handle_close: [%Q{}, _, :newest_state], + handle_commit: [_, :state2] + ] = A.record(agent) + end + test "stream returns result" do stack = [ {:ok, :state}, @@ -70,4 +416,305 @@ defmodule StreamStageTest do handle_commit: [_, :newer_state2] ] = A.record(agent) end + + test "stream notifies new consumers that it's done" do + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, %C{}, :newer_state}, + {:deallocate, %R{}, :state2}, + {:ok, :deallocated, :new_state2}, + {:ok, :commited, :newer_state2} + ] + {:ok, agent} = A.start_link(stack) + + parent = self() + opts = [agent: agent, parent: parent] + {:ok, pool} = P.start_link(opts) + {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) + + ref = make_ref() + send(stage, {:"$gen_producer", {parent, ref}, {:subscribe, nil, []}}) + sub = {stage, ref} + GenStage.ask(sub, 1000) + + assert_receive {:"$gen_consumer", ^sub, [%R{}]} + assert_receive {:"$gen_consumer", ^sub, {:notification, {:producer, :halted}}} + + assert stage |> Flow.from_stage() |> Enum.to_list() == [] + + mon = Process.monitor(stage) + GenStage.cancel(sub, :normal) + assert_receive {:DOWN, ^mon, :process, ^stage, :normal} + assert_received {:"$gen_consumer", ^sub, {:cancel, :normal}} + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_declare: [%Q{}, [:param], _, :new_state], + handle_first: [%Q{}, %C{}, _, :newer_state], + handle_deallocate: [%Q{}, %C{}, _, :state2], + handle_commit: [_, :new_state2] + ] = A.record(agent) + end + + test "stream finishes when consumers do" do + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, %C{}, :newer_state}, + {:ok, :deallocated, :newest_state}, + {:ok, :commited, :state2} + ] + {:ok, agent} = A.start_link(stack) + + parent = self() + opts = [agent: agent, parent: parent] + {:ok, pool} = P.start_link(opts) + {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) + + ref = make_ref() + send(stage, {:"$gen_producer", {parent, ref}, {:subscribe, nil, []}}) + sub = {stage, ref} + + mon = Process.monitor(stage) + GenStage.cancel(sub, :normal) + assert_receive {:DOWN, ^mon, :process, ^stage, :normal} + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_declare: [%Q{}, [:param], _, :new_state], + handle_deallocate: [%Q{}, %C{}, _, :newer_state], + handle_commit: [_, :newest_state] + ] = A.record(agent) + end + + test "stream rolls back on abnormal exit" do + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, %C{}, :newer_state}, + {:ok, :deallocated, :newest_state}, + {:ok, :rolledback, :state2} + ] + {:ok, agent} = A.start_link(stack) + + parent = self() + opts = [agent: agent, parent: parent] + Process.flag(:trap_exit, true) + {:ok, pool} = P.start_link(opts) + {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) + + send(stage, {:"$gen_producer", {parent, make_ref()}, {:subscribe, nil, []}}) + + GenStage.stop(stage, :oops) + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_declare: [%Q{}, [:param], _, :new_state], + handle_deallocate: [%Q{}, %C{}, _, :newer_state], + handle_rollback: [_, :newest_state] + ] = A.record(agent) + end + + test "stream declare disconnects" do + err = RuntimeError.exception("oops") + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:disconnect, err, :newer_state}, + :ok, + fn(opts) -> + send(opts[:parent], :reconnected) + {:ok, :state2} + end + ] + {:ok, agent} = A.start_link(stack) + + parent = self() + opts = [agent: agent, parent: parent] + Process.flag(:trap_exit, true) + {:ok, pool} = P.start_link(opts) + assert {:error, {^err, _}} = P.stream_stage(pool, %Q{}, [:param], opts) + + assert_receive :reconnected + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_declare: [%Q{}, [:param], _, :new_state], + disconnect: [^err, :newer_state], + connect: [_] + ] = A.record(agent) + end + + test "stream declare bad return raises and stops" do + stack = [ + fn(opts) -> + send(opts[:parent], {:hi, self()}) + Process.link(opts[:parent]) + {:ok, :state} + end, + {:ok, :began, :new_state}, + :oops, + {:ok, :state2} + ] + {:ok, agent} = A.start_link(stack) + + parent = self() + opts = [agent: agent, parent: parent] + {:ok, pool} = P.start_link(opts) + assert_receive {:hi, conn} + + Process.flag(:trap_exit, true) + {:error, {%DBConnection.ConnectionError{}, _} = reason} = P.stream_stage(pool, %Q{}, [:param], opts) + + assert_receive {:EXIT, stage, ^reason} + + prefix = "client #{inspect stage} stopped: " <> + "** (DBConnection.ConnectionError) bad return value: :oops" + len = byte_size(prefix) + assert_receive {:EXIT, ^conn, + {%DBConnection.ConnectionError{message: <<^prefix::binary-size(len), _::binary>>}, + [_|_]}} + + assert [ + {:connect, _}, + {:handle_begin, [_, :state]}, + {:handle_declare, [%Q{}, [:param], _, :new_state]} | _] = A.record(agent) + end + + test "stream rolls back if first errors" do + err = RuntimeError.exception("oops") + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, %C{}, :newer_state}, + {:error, err, :newest_state}, + {:ok, :deallocated, :state2}, + {:ok, :rolledback, :new_state2} + ] + {:ok, agent} = A.start_link(stack) + + parent = self() + opts = [agent: agent, parent: parent] + {:ok, pool} = P.start_link(opts) + Process.flag(:trap_exit, true) + {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) + catch_exit(stage |> Flow.from_stage() |> Enum.to_list()) + assert_receive {:EXIT, ^stage, {^err, _}} + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_declare: [%Q{}, [:param], _, :new_state], + handle_first: [%Q{}, %C{}, _, :newer_state], + handle_deallocate: [%Q{}, %C{}, _, :newest_state], + handle_rollback: [_, :state2] + ] = A.record(agent) + end + + test "stream first disconnects" do + err = RuntimeError.exception("oops") + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, %C{}, :newer_state}, + {:disconnect, err, :newest_state}, + :ok, + fn(opts) -> + send(opts[:parent], :reconnected) + {:ok, :state2} + end + ] + {:ok, agent} = A.start_link(stack) + + parent = self() + opts = [agent: agent, parent: parent] + {:ok, pool} = P.start_link(opts) + Process.flag(:trap_exit, true) + {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) + catch_exit(stage |> Flow.from_stage() |> Enum.to_list()) + assert_receive {:EXIT, ^stage, {^err, _}} + + assert_receive :reconnected + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_declare: [%Q{}, [:param], _, :new_state], + handle_first: [%Q{}, %C{}, _, :newer_state], + disconnect: [^err, :newest_state], + connect: [_] + ] = A.record(agent) + end + + test "stream rolls back if deallocate errors" do + err = RuntimeError.exception("oops") + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, %C{}, :newer_state}, + {:deallocate, %R{}, :newest_state}, + {:error, err, :state2}, + {:ok, :rolledback, :new_state2} + ] + {:ok, agent} = A.start_link(stack) + + parent = self() + opts = [agent: agent, parent: parent] + {:ok, pool} = P.start_link(opts) + Process.flag(:trap_exit, true) + {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) + catch_exit(stage |> Flow.from_stage() |> Enum.to_list()) + assert_receive {:EXIT, ^stage, {^err, _}} + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_declare: [%Q{}, [:param], _, :new_state], + handle_first: [%Q{}, %C{}, _, :newer_state], + handle_deallocate: [%Q{}, %C{}, _, :newest_state], + handle_rollback: [_, :state2] + ] = A.record(agent) + end + + test "stream deallocate disconnects" do + err = RuntimeError.exception("oops") + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, %C{}, :newer_state}, + {:deallocate, %R{}, :newest_state}, + {:disconnect, err, :state2}, + :ok, + fn(opts) -> + send(opts[:parent], :reconnected) + {:ok, :new_state2} + end + ] + {:ok, agent} = A.start_link(stack) + + parent = self() + opts = [agent: agent, parent: parent] + {:ok, pool} = P.start_link(opts) + Process.flag(:trap_exit, true) + {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) + catch_exit(stage |> Flow.from_stage() |> Enum.to_list()) + assert_receive {:EXIT, ^stage, {^err, _}} + + assert_receive :reconnected + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_declare: [%Q{}, [:param], _, :new_state], + handle_first: [%Q{}, %C{}, _, :newer_state], + handle_deallocate: [%Q{}, %C{}, _, :newest_state], + disconnect: [^err, :state2], + connect: [_] + ] = A.record(agent) + end end diff --git a/lib/db_connection/stage.ex b/lib/db_connection/stage.ex index d9273a6f..a12f3efd 100644 --- a/lib/db_connection/stage.ex +++ b/lib/db_connection/stage.ex @@ -188,11 +188,20 @@ defmodule DBConnection.Stage do end @doc false - def handle_subscribe(:producer, _, {pid, ref}, stage) do - %Stage{producers: producers, active: active} = stage - stage = %Stage{stage | producers: Map.put(producers, ref, pid), - active: [ref | active]} - {:automatic, stage} + def handle_subscribe(:producer, _, {pid, ref} = from, stage) do + case stage do + %Stage{done?: true, type: :consumer, producers: producers} -> + GenStage.cancel(from, :normal, [:noconnect]) + stage = %Stage{stage | producers: Map.put(producers, ref, pid)} + {:manual, stage} + %Stage{done?: true, type: :producer_consumer, producers: producers} -> + stage = %Stage{stage | producers: Map.put(producers, ref, pid)} + {:manual, stage} + %Stage{done?: false, producers: producers, active: active} -> + stage = %Stage{stage | producers: Map.put(producers, ref, pid), + active: [ref | active]} + {:automatic, stage} + end end def handle_subscribe(:consumer, _, {_, ref}, stage) do %Stage{consumers: consumers} = stage @@ -329,6 +338,7 @@ defmodule DBConnection.Stage do end end defp stream_next(_, _, state) when state in [:halted, :done] do + GenStage.async_notify(self(), {:producer, state}) {[], state} end diff --git a/test/test_support.exs b/test/test_support.exs index 74431acd..7549796b 100644 --- a/test/test_support.exs +++ b/test/test_support.exs @@ -63,6 +63,11 @@ defmodule TestConnection do DBConnection.close!(pool, query, opts2 ++ unquote(opts)) end + def stage(pool, type, start, handle, stop, opts2 \\ []) do + opts3 = opts2 ++ unquote(opts) + DBConnection.Stage.start_link(pool, type, start, handle, stop, opts3) + end + def stream_stage(pool, query, params, opts2 \\ []) do DBConnection.Stage.stream(pool, query, params, opts2 ++ unquote(opts)) end From 392affacc4a730d748bf95d7261de9e8a9709fe2 Mon Sep 17 00:00:00 2001 From: James Fish Date: Mon, 10 Apr 2017 19:59:38 +0100 Subject: [PATCH 04/20] Fix stage docs --- lib/db_connection/stage.ex | 24 ++++++++++++++++-------- 1 file changed, 16 insertions(+), 8 deletions(-) diff --git a/lib/db_connection/stage.ex b/lib/db_connection/stage.ex index a12f3efd..0cffbea3 100644 --- a/lib/db_connection/stage.ex +++ b/lib/db_connection/stage.ex @@ -40,8 +40,8 @@ defmodule DBConnection.Stage do ### Example - query = %Query{statement: "SELECT id FROM table"} - {:ok, stage} = DBConnection.StreamStage.prepare_stream_link(conn, query, []) + query = %Query{statement: "SELECT id FROM table"} + {:ok, stage} = DBConnection.Stage.prepare_stream(pool, query, []) stage |> Flow.from_stage() |> Enum.to_list() end) """ @@ -76,8 +76,8 @@ defmodule DBConnection.Stage do ### Example - query = %Query{statement: "SELECT id FROM table"} - {:ok, stage} = DBConnection.StreamStage.stream_link(conn, query, []) + query = DBConnection.prepare!(pool, %Query{statement: "SELECT id FROM table"}) + {:ok, stage} = DBConnection.Stage.stream(pool, query, []) stage |> Flow.from_stage() |> Enum.to_list() end) """ @@ -119,8 +119,10 @@ defmodule DBConnection.Stage do * It will stop with reason `:normal` when the last consumer cancels * It will notify consumers that it is done when all producers have cancelled or notified that they are done or halted - * It will cancel all remaining producers when all producers have notified - that they are done or halted + * It will cancel new and remaining producers when all producers have + notified that they are done or halted and it is a `:consumer` + * It will not send demand to new producers when all producers have notified + that they are done or halted and it is a `:consumer_producer` ### Options @@ -143,12 +145,18 @@ defmodule DBConnection.Stage do ### Example - start = &DBConnection.prepare!(&1, %2, opts) + start = + fn(conn) -> + DBConnection.prepare!(conn, query, opts) + end handle = fn(conn, param, query) -> {[DBConection.execute!(conn, param, query, opts)], query} end - stop = %DBConnection.close(&1, &2, opts) + stop = + fn(conn, _, query) -> + DBConnection.close!(conn, query, opts) + end DBConnection.Stage.start_link(pool, :producer_consumer, start, handle, stop, opts) """ From 92c45d3fec04c198eb1d7c304c9e49061e912f0a Mon Sep 17 00:00:00 2001 From: James Fish Date: Tue, 11 Apr 2017 13:57:23 +0100 Subject: [PATCH 05/20] Split Stage.start_link into 3 separate functions --- integration_test/cases/stage_test.exs | 18 +- lib/db_connection/stage.ex | 276 +++++++++++++++++--------- test/test_support.exs | 17 +- 3 files changed, 205 insertions(+), 106 deletions(-) diff --git a/integration_test/cases/stage_test.exs b/integration_test/cases/stage_test.exs index 7ab27f4f..e09ac940 100644 --- a/integration_test/cases/stage_test.exs +++ b/integration_test/cases/stage_test.exs @@ -26,7 +26,7 @@ defmodule StageTest do {[DBConnection.execute!(conn, query, [demand])], query} end stop = fn(conn, _, query) -> DBConnection.close!(conn, query) end - {:ok, stage} = P.stage(pool, :producer, start, handle, stop, opts) + {:ok, stage} = P.producer(pool, start, handle, stop, opts) mon = Process.monitor(stage) assert stage |> Flow.from_stage() |> Enum.to_list() == [%R{}] @@ -62,7 +62,7 @@ defmodule StageTest do {[DBConnection.execute!(conn, query, [demand])], query} end stop = fn(conn, _, query) -> DBConnection.close!(conn, query) end - {:ok, stage} = P.stage(pool, :producer, start, handle, stop, opts) + {:ok, stage} = P.producer(pool, start, handle, stop, opts) ref = make_ref() sub_opts = [cancel: :temporary] @@ -101,7 +101,7 @@ defmodule StageTest do DBConnection.rollback(conn, :normal) end stop = fn(conn, _, query) -> DBConnection.close!(conn, query) end - {:ok, stage} = P.stage(pool, :producer, start, handle, stop, opts) + {:ok, stage} = P.producer(pool, start, handle, stop, opts) mon = Process.monitor(stage) catch_exit(stage |> Flow.from_stage() |> Enum.to_list()) @@ -134,7 +134,7 @@ defmodule StageTest do {[:oops], query} end stop = fn(conn, _, query) -> DBConnection.close!(conn, query) end - {:ok, stage} = P.stage(pool, :producer, start, handle, stop, opts) + {:ok, stage} = P.producer(pool, start, handle, stop, opts) mon = Process.monitor(stage) assert stage |> Flow.from_stage() |> Enum.take(1) == [:oops] @@ -165,7 +165,7 @@ defmodule StageTest do :oops end fail = fn(_, _, _) -> flunk "should not run" end - {:ok, stage} = P.stage(pool, :producer, start, fail, fail, opts) + {:ok, stage} = P.producer(pool, start, fail, fail, opts) Process.flag(:trap_exit, true) catch_exit(stage |> Flow.from_stage() |> Enum.to_list()) @@ -197,7 +197,7 @@ defmodule StageTest do {[], query} end stop = fn(conn, _, query) -> DBConnection.close!(conn, query) end - {:ok, stage} = P.stage(pool, :consumer, start, handle, stop, opts) + {:ok, stage} = P.consumer(pool, start, handle, stop, opts) mon = Process.monitor(stage) flow = Flow.from_enumerable([:param]) {:ok, _} = Flow.into_stages(flow, [stage]) @@ -234,7 +234,7 @@ defmodule StageTest do {[], query} end stop = fn(conn, _, query) -> DBConnection.close!(conn, query) end - {:ok, stage} = P.stage(pool, :consumer, start, handle, stop, opts) + {:ok, stage} = P.consumer(pool, start, handle, stop, opts) :ok = GenStage.async_subscribe(stage, [to: self(), cancel: :temporary, max_demand: 1]) @@ -281,7 +281,7 @@ defmodule StageTest do {[DBConnection.execute!(conn, query, events)], query} end stop = fn(conn, _, query) -> DBConnection.close!(conn, query) end - {:ok, stage} = P.stage(pool, :producer_consumer, start, handle, stop, opts) + {:ok, stage} = P.producer_consumer(pool, start, handle, stop, opts) mon = Process.monitor(stage) {:ok, _} = Flow.from_enumerable([:param]) |> Flow.into_stages([stage]) assert Flow.from_stage(stage) |> Enum.to_list() == [%R{}] @@ -317,7 +317,7 @@ defmodule StageTest do {[DBConnection.execute!(conn, query, events)], query} end stop = fn(conn, _, query) -> DBConnection.close!(conn, query) end - {:ok, stage} = P.stage(pool, :producer_consumer, start, handle, stop, opts) + {:ok, stage} = P.producer_consumer(pool, start, handle, stop, opts) :ok = GenStage.async_subscribe(stage, [to: self(), cancel: :temporary, max_demand: 1]) diff --git a/lib/db_connection/stage.ex b/lib/db_connection/stage.ex index 0cffbea3..41364750 100644 --- a/lib/db_connection/stage.ex +++ b/lib/db_connection/stage.ex @@ -1,6 +1,22 @@ defmodule DBConnection.Stage do @moduledoc """ A `GenStage` process that encapsulates a transaction. + + ### Options + + * `:pool_timeout` - The maximum time to wait for a reply when making a + synchronous call to the pool (default: `5_000`) + * `:queue` - Whether to block waiting in an internal queue for the + connection's state (boolean, default: `true`) + * `:timeout` - The maximum time that the caller is allowed the + to hold the connection's state (ignored when using a run/transaction + connection, default: `15_000`) + * `:log` - A function to log information about a call, either + a 1-arity fun, `{module, function, args}` with `DBConnection.LogEntry.t` + prepended to `args` or `nil`. See `DBConnection.LogEntry` (default: `nil`) + + The pool and connection module may support other options. All options + are passed to `handle_begin/2`, `handle_commit/2` and `handle_rollback/2`. """ alias __MODULE__, as: Stage @@ -20,23 +36,8 @@ defmodule DBConnection.Stage do The transaction is rolled back if the process terminates with a reason other than `:normal`. - ### Options - - * `:pool_timeout` - The maximum time to wait for a reply when making a - synchronous call to the pool (default: `5_000`) - * `:queue` - Whether to block waiting in an internal queue for the - connection's state (boolean, default: `true`) - * `:timeout` - The maximum time that the caller is allowed the - to hold the connection's state (ignored when using a run/transaction - connection, default: `15_000`) - * `:log` - A function to log information about a call, either - a 1-arity fun, `{module, function, args}` with `DBConnection.LogEntry.t` - prepended to `args` or `nil`. See `DBConnection.LogEntry` (default: `nil`) - - The pool and connection module may support other options. All options - are passed to `handle_begin/2`, `handle_prepare/3, `handle_close/3, - `handle_declare/4`, `handle_first/4`, `handle_next/4`, `handle_deallocate/4`, - `handle_commit/2` and `handle_rollback/2`. + All options are passed to `DBConnection.prepare_stream/4`. For options see + "Options" in the module documentation. ### Example @@ -56,23 +57,8 @@ defmodule DBConnection.Stage do The transaction is rolled back if the process terminates with a reason other than `:normal`. - ### Options - - * `:pool_timeout` - The maximum time to wait for a reply when making a - synchronous call to the pool (default: `5_000`) - * `:queue` - Whether to block waiting in an internal queue for the - connection's state (boolean, default: `true`) - * `:timeout` - The maximum time that the caller is allowed the - to hold the connection's state (ignored when using a run/transaction - connection, default: `15_000`) - * `:log` - A function to log information about a call, either - a 1-arity fun, `{module, function, args}` with `DBConnection.LogEntry.t` - prepended to `args` or `nil`. See `DBConnection.LogEntry` (default: `nil`) - - The pool and connection module may support other options. All options - are passed to `handle_begin/2`, `handle_declare/4`, `handle_first/4`, - `handle_next/4`, `handle_deallocate/4`, `handle_commit/2` and - `handle_rollback/2`. + All options are passed to `DBConnection.stream/4`. For options see "Options" + in the module documentation ### Example @@ -86,62 +72,167 @@ defmodule DBConnection.Stage do end @doc """ - Start link a `GenStage` process that will run a transaction for its duration. + Start link a `GenStage` producer that will run a transaction for its duration. - The first argument is the pool, the second argument is the `GenStage` type, - the third argument is the start function, the fourth argument is the handle - function, the fifth argument is the stop function and the optional sixth - argument are the options. + The first argument is the pool, the second argument is the start function, + the third argument is the handle demand function, the fourth argument is the + stop function and the optional fiftth argument are the options. The start function is a 1-arity anonymous function with argument `DBConnection.t`. This is called after the transaction begins but before - `start_link/6` returns. It should return the `state` or call + `producer/5` returns. It should return the accumulator or call `DBConnection.rollback/2` to stop the `GenStage`. - The handle function is a 3-arity anonymous function. The first argument is the - `DBConnection.t` for the transaction and the third argument is the state. - If the `GenStage` type is a `:producer`, then the second argument is the - `demand` from a `GenStage` `handle_demand` callback. Otherwise the second - argument is the events from a `GenStage` `handle_events` callback. This - function returns a 2-tuple, with first element as events (empty list for - `:consumer`) and second element as the `state`. This function can roll back + The handle demand function is a 3-arity anonymous function. The first argument + is the `DBConnection.t` for the transaction, the second argument is the + `demand`, and the third argument is the accumulator. This function returns a + 2-tuple, with first element as list of events to fulfil the demand and second + element as the accumulator. If the producer has emitted all events (and so + not fulfilled demand) it should call + `GenStage.async_notify(self(), {:producer, :done | :halted}` to signal to + consumers that it has finished. Also this function can rollback and stop the + `GenStage` using `DBConnection.rollback/2`. + + The stop function is a 3-arity anonymous function. The first argument is the + `DBConnection.t` for the transaction, the second argument is the terminate + reason and the third argument is the accumulator. This function will only be + called if connection is alive and the transaction has not been rolled back. If + this function returns the transaction is committed. This function can rollback and stop the `GenStage` using `DBConnection.rollback/2`. + For options see "Options" in the module documentation. + + The `GenStage` process will behave like a `Flow` stage: + + * It will stop with reason `:normal` when the last consumer cancels + + ### Example + + start = + fn(conn) -> + {ids, DBConnection.prepare!(conn, query, opts)} + end + handle_demand = + fn(conn, demand, {ids, query}) -> + {param, rem} = Enum.split(ids, demand) + %{rows: rows} = DBConection.execute!(conn, query, [param]) + if rem == [], do: GenStage.async_notify(self(), {:producer, :done}) + {rows, {rem, query}} + end + stop = + fn(conn, reason, {rem, query}) -> + DBConnection.close!(conn, query, opts) + if reason != :normal or rem != [] do + DBConnection.rollback(conn, reason) + end + end + DBConnection.Stage.producer(pool, start, handle_demand, stop) + """ + @spec producer(GenServer.server, ((DBConnection.t) -> acc), + ((DBConnection.t, demand :: pos_integer, acc) -> {[any], acc}), + ((DBConnection.t, reason :: any, acc) -> any), Keyword.t) :: + GenServer.on_start when acc: var + def producer(pool, start, handle_demand, stop, opts \\ []) do + start_link(pool, :producer, start, handle_demand, stop, opts) + end + + @doc """ + Start link a `GenStage` producer consumer that will run a transaction for its + duration. + + The first argument is the pool, the second argument is the start function, + the third argument is the handle events function, the fourth argument is the + stop function and the optional fiftth argument are the options. + + The start function is a 1-arity anonymous function with argument + `DBConnection.t`. This is called after the transaction begins but before + `producer_consumer/5` returns. It should return the accumulator or call + `DBConnection.rollback/2` to stop the `GenStage`. + + The handle events function is a 3-arity anonymous function. The first argument + is the `DBConnection.t` for the transaction, the second argument is a list of + incoming events, and the third argument is the accumulator. This function + returns a 2-tuple, with first element as list of outgoing events and second + element as the accumulator. Also this function can rollback and stop the + `GenStage` using `DBConnection.rollback/2`. + The stop function is a 3-arity anonymous function. The first argument is the `DBConnection.t` for the transaction, the second argument is the terminate - reason and the third argument is the `state`. This function will only be + reason and the third argument is the accumulator. This function will only be called if connection is alive and the transaction has not been rolled back. If - this function returns the transaction is commited. This function can roll back + this function returns the transaction is committed. This function can rollback and stop the `GenStage` using `DBConnection.rollback/2`. + For options see "Options" in the module documentation. + The `GenStage` process will behave like a `Flow` stage: * It will stop with reason `:normal` when the last consumer cancels * It will notify consumers that it is done when all producers have cancelled or notified that they are done or halted - * It will cancel new and remaining producers when all producers have - notified that they are done or halted and it is a `:consumer` * It will not send demand to new producers when all producers have notified - that they are done or halted and it is a `:consumer_producer` + that they are done or halted - ### Options + ### Example - * `:name` - A name to register the started process (see the `:name` option - in `GenServer.start_link/3`) - * `:pool_timeout` - The maximum time to wait for a reply when making a - synchronous call to the pool (default: `5_000`) - * `:queue` - Whether to block waiting in an internal queue for the - connection's state (boolean, default: `true`) - * `:timeout` - The maximum time that the caller is allowed the - to hold the connection's state (ignored when using a run/transaction - connection, default: `15_000`) - * `:log` - A function to log information about a call, either - a 1-arity fun, `{module, function, args}` with `DBConnection.LogEntry.t` - prepended to `args` or `nil`. See `DBConnection.LogEntry` (default: `nil`) + start = + fn(conn) -> + DBConnection.prepare!(conn, query, opts) + end + handle_events = + fn(conn, ids, query) -> + %{rows: rows} = DBConection.execute!(conn, query, [ids]) + {rows, query} + end + stop = + fn(conn, reason, query) -> + DBConnection.close!(conn, query, opts) + if reason != :normal do + DBConnection.rollback(conn, reason) + end + end + DBConnection.Stage.consumer_producer(pool, start, handle_events, stop) + """ + @spec producer_consumer(GenServer.server, ((DBConnection.t) -> acc), + ((DBConnection.t, events_in :: [any], acc) -> {events_out :: [any], acc}), + ((DBConnection.t, reason :: any, acc) -> any), Keyword.t) :: + GenServer.on_start when acc: var + def producer_consumer(pool, start, handle_events, stop, opts \\ []) do + start_link(pool, :producer_consumer, start, handle_events, stop, opts) + end - The pool and connection module may support other options. All options - are passed to `handle_begin/2`, `handle_commit/2` and `handle_rollback/2`. - All options are passed to the `GenStage` on init. + @doc """ + Start link a `GenStage` consumer that will run a transaction for its duration. + + The first argument is the pool, the second argument is the start function, + the third argument is the handle events function, the fourth argument is the + stop function and the optional fiftth argument are the options. + + The start function is a 1-arity anonymous function with argument + `DBConnection.t`. This is called after the transaction begins but before + `consumer/5` returns. It should return the accumulator or call + `DBConnection.rollback/2` to stop the `GenStage`. + + The handle events function is a 3-arity anonymous function. The first argument + is the `DBConnection.t` for the transaction, the second argument is the + events, and the third argument is the accumulator. This function returns a + 2-tuple, with first element is an empty list (as no outgoing events) and + second element as the accumulator. Also this function can rollback and stop + the `GenStage` using `DBConnection.rollback/2`. + + The stop function is a 3-arity anonymous function. The first argument is the + `DBConnection.t` for the transaction, the second argument is the terminate + reason and the third argument is the accumulator. This function will only be + called if connection is alive and the transaction has not been rolled back. If + this function returns the transaction is committed. This function can rollback + and stop the `GenStage` using `DBConnection.rollback/2`. + + For options see "Options" in the module documentation. + + The `GenStage` process will behave like a `Flow` stage: + + * It will cancel new and remaining producers when all producers have + notified that they are done or halted and it is a `:consumer` ### Example @@ -149,36 +240,26 @@ defmodule DBConnection.Stage do fn(conn) -> DBConnection.prepare!(conn, query, opts) end - handle = - fn(conn, param, query) -> - {[DBConection.execute!(conn, param, query, opts)], query} + handle_events = + fn(conn, ids, query) -> + DBConection.execute!(conn, query, [ids]) + {[], query} end stop = - fn(conn, _, query) -> + fn(conn, reason, query) -> DBConnection.close!(conn, query, opts) + if reason != :normal do + DBConnection.rollback(conn, reason) + end end - DBConnection.Stage.start_link(pool, :producer_consumer, - start, handle, stop, opts) + DBConnection.Stage.consumer(pool, start, handle_events, stop) """ - @spec start_link(GenServer.server, :producer, - ((DBConnection.t) -> state), - ((DBConnection.t, demand :: pos_integer, state) -> {[any], state}), - ((DBConnection.t, reason :: any, state) -> any), Keyword.t) :: - GenServer.on_start when state: var - @spec start_link(GenServer.server, :producer_consumer, - ((DBConnection.t) -> state), - ((DBConnection.t, [any], state) -> {[any], state}), - ((DBConnection.t, reason :: any, state) -> any), Keyword.t) :: - GenServer.on_start when state: var - @spec start_link(GenServer.server, :consumer, - ((DBConnection.t) -> state), - ((DBConnection.t, [any], state) -> {[], state}), - ((DBConnection.t, reason :: any, state) -> any), Keyword.t) :: - GenServer.on_start when state: var - def start_link(pool, type, start, handle, stop, opts \\ []) do - start_opts = Keyword.take(opts, @start_opts) - args = {pool, type, start, handle, stop, opts} - GenStage.start_link(__MODULE__, args, start_opts) + @spec consumer(GenServer.server, ((DBConnection.t) -> acc), + ((DBConnection.t, events_in :: [any], acc) -> {[], acc}), + ((DBConnection.t, reason :: any, acc) -> any), Keyword.t) :: + GenServer.on_start when acc: var + def consumer(pool, start, handle_events, stop, opts \\ []) do + start_link(pool, :consumer, start, handle_events, stop, opts) end @doc false @@ -245,6 +326,7 @@ defmodule DBConnection.Stage do end end + @doc false def handle_info({{_, ref}, {:producer, state}}, stage) when state in [:halted, :done] do %Stage{type: type, producers: producers, active: active, done?: done?} = stage @@ -320,7 +402,13 @@ defmodule DBConnection.Stage do defp stream(pool, stream_fun, query, params, opts) do start = &stream_start(&1, stream_fun, query, params, opts) - start_link(pool, :producer, start, &stream_next/3, &stream_stop/3, opts) + producer(pool, start, &stream_next/3, &stream_stop/3, opts) + end + + defp start_link(pool, type, start, handle, stop, opts) do + start_opts = Keyword.take(opts, @start_opts) + args = {pool, type, start, handle, stop, opts} + GenStage.start_link(__MODULE__, args, start_opts) end defp stream_start(conn, stream_fun, query, params, opts) do diff --git a/test/test_support.exs b/test/test_support.exs index 7549796b..2e9ab30f 100644 --- a/test/test_support.exs +++ b/test/test_support.exs @@ -63,9 +63,19 @@ defmodule TestConnection do DBConnection.close!(pool, query, opts2 ++ unquote(opts)) end - def stage(pool, type, start, handle, stop, opts2 \\ []) do + def producer(pool, start, handle, stop, opts2 \\ []) do opts3 = opts2 ++ unquote(opts) - DBConnection.Stage.start_link(pool, type, start, handle, stop, opts3) + DBConnection.Stage.producer(pool, start, handle, stop, opts3) + end + + def producer_consumer(pool, start, handle, stop, opts2 \\ []) do + opts3 = opts2 ++ unquote(opts) + DBConnection.Stage.producer_consumer(pool, start, handle, stop, opts3) + end + + def consumer(pool, start, handle, stop, opts2 \\ []) do + opts3 = opts2 ++ unquote(opts) + DBConnection.Stage.consumer(pool, start, handle, stop, opts3) end def stream_stage(pool, query, params, opts2 \\ []) do @@ -73,7 +83,8 @@ defmodule TestConnection do end def prepare_stream_stage(pool, query, params, opts2 \\ []) do - DBConnection.Stage.prepare_stream(pool, query, params, opts2 ++ unquote(opts)) + opts3 = opts2 ++ unquote(opts) + DBConnection.Stage.prepare_stream(pool, query, params, opts3) end defoverridable [start_link: 1] From 73238baf9778b1263e108a0df9198aade2921bea Mon Sep 17 00:00:00 2001 From: James Fish Date: Tue, 11 Apr 2017 14:00:19 +0100 Subject: [PATCH 06/20] Notify late consumers if producer_consumer is done --- integration_test/cases/stage_test.exs | 59 ++++++++++++++++++++++++++- lib/db_connection/stage.ex | 5 ++- 2 files changed, 60 insertions(+), 4 deletions(-) diff --git a/integration_test/cases/stage_test.exs b/integration_test/cases/stage_test.exs index e09ac940..b68da9b5 100644 --- a/integration_test/cases/stage_test.exs +++ b/integration_test/cases/stage_test.exs @@ -115,7 +115,7 @@ defmodule StageTest do ] = A.record(agent) end - test "producer does not stop or commit on nested rollback" do + test "producer does not stop or commit on handle nested rollback" do stack = [ {:ok, :state}, {:ok, :began, :new_state}, @@ -148,7 +148,7 @@ defmodule StageTest do ] = A.record(agent) end - test "producer exits but does not stop or commit after nested rollback" do + test "producer exits but does not commit on start nested rollback" do stack = [ {:ok, :state}, {:ok, :began, :new_state}, @@ -353,6 +353,61 @@ defmodule StageTest do ] = A.record(agent) end + test "producer_consumer does notifies new consumers after done" do + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, %Q{}, :newer_state}, + {:ok, %R{}, :newest_state}, + {:ok, :closed, :state2}, + {:ok, :commited, :new_state2} + ] + {:ok, agent} = A.start_link(stack) + + parent = self() + opts = [agent: agent, parent: parent] + {:ok, pool} = P.start_link(opts) + start = &DBConnection.prepare!(&1, %Q{}) + handle = fn(conn, events, query) -> + {[DBConnection.execute!(conn, query, events)], query} + end + stop = fn(conn, _, query) -> DBConnection.close!(conn, query) end + {:ok, stage} = P.producer_consumer(pool, start, handle, stop, opts) + + mon = Process.monitor(stage) + + {:ok, coord} = [:param] |> Flow.from_enumerable() |> Flow.into_stages([stage]) + + send(stage, {:"$gen_producer", {self(), mon}, {:subscribe, nil, []}}) + send(stage, {:"$gen_producer", {self(), mon}, {:ask, 1000}}) + + sub1 = {stage, mon} + + assert_receive {:"$gen_consumer", ^sub1, [%R{}]} + assert_receive {:"$gen_consumer", ^sub1, {:notification, {:producer, :done}}} + + task = Task.async(fn() -> + [stage] |> Flow.from_stages() |> Enum.to_list() + end) + + :timer.sleep(100) + + GenStage.cancel(sub1, :normal) + + assert Task.await(task) == [] + + assert_receive {:DOWN, ^mon, _, _, :normal} + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_prepare: [%Q{}, _, :new_state], + handle_execute: [%Q{}, [:param], _, :newer_state], + handle_close: [%Q{}, _, :newest_state], + handle_commit: [_, :state2] + ] = A.record(agent) + end + test "stream returns result" do stack = [ {:ok, :state}, diff --git a/lib/db_connection/stage.ex b/lib/db_connection/stage.ex index 41364750..1df9f8f1 100644 --- a/lib/db_connection/stage.ex +++ b/lib/db_connection/stage.ex @@ -292,8 +292,9 @@ defmodule DBConnection.Stage do {:automatic, stage} end end - def handle_subscribe(:consumer, _, {_, ref}, stage) do - %Stage{consumers: consumers} = stage + def handle_subscribe(:consumer, _, {pid, ref}, stage) do + %Stage{done?: done?, consumers: consumers} = stage + if done?, do: send(pid, {{self(), ref}, {:producer, :done}}) {:automatic, %Stage{stage | consumers: [ref | consumers]}} end From 389e7eb401b8b8f03bd44a14980b0866e52f7b18 Mon Sep 17 00:00:00 2001 From: James Fish Date: Tue, 11 Apr 2017 23:32:39 +0100 Subject: [PATCH 07/20] Send async_notify when done but consumer subscribes --- integration_test/cases/stage_test.exs | 2 +- lib/db_connection/stage.ex | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/integration_test/cases/stage_test.exs b/integration_test/cases/stage_test.exs index b68da9b5..ab43e51c 100644 --- a/integration_test/cases/stage_test.exs +++ b/integration_test/cases/stage_test.exs @@ -376,7 +376,7 @@ defmodule StageTest do mon = Process.monitor(stage) - {:ok, coord} = [:param] |> Flow.from_enumerable() |> Flow.into_stages([stage]) + {:ok, _} = [:param] |> Flow.from_enumerable() |> Flow.into_stages([stage]) send(stage, {:"$gen_producer", {self(), mon}, {:subscribe, nil, []}}) send(stage, {:"$gen_producer", {self(), mon}, {:ask, 1000}}) diff --git a/lib/db_connection/stage.ex b/lib/db_connection/stage.ex index 1df9f8f1..cb15087f 100644 --- a/lib/db_connection/stage.ex +++ b/lib/db_connection/stage.ex @@ -292,9 +292,9 @@ defmodule DBConnection.Stage do {:automatic, stage} end end - def handle_subscribe(:consumer, _, {pid, ref}, stage) do + def handle_subscribe(:consumer, _, {_, ref}, stage) do %Stage{done?: done?, consumers: consumers} = stage - if done?, do: send(pid, {{self(), ref}, {:producer, :done}}) + if done?, do: GenStage.async_notify(self(), {:producer, :done}) {:automatic, %Stage{stage | consumers: [ref | consumers]}} end From a34057957dff2252340684959deb211e81c577a5 Mon Sep 17 00:00:00 2001 From: James Fish Date: Wed, 24 May 2017 14:53:46 -0400 Subject: [PATCH 08/20] Update GenStage and Flow --- integration_test/cases/stage_test.exs | 87 +++++++++++++-------------- lib/db_connection/stage.ex | 79 ++++++------------------ mix.exs | 6 +- mix.lock | 4 +- 4 files changed, 68 insertions(+), 108 deletions(-) diff --git a/integration_test/cases/stage_test.exs b/integration_test/cases/stage_test.exs index ab43e51c..5cb529ef 100644 --- a/integration_test/cases/stage_test.exs +++ b/integration_test/cases/stage_test.exs @@ -22,7 +22,7 @@ defmodule StageTest do {:ok, pool} = P.start_link(opts) start = &DBConnection.prepare!(&1, %Q{}) handle = fn(conn, demand, query) -> - GenStage.async_notify(self(), {:producer, :done}) + GenStage.async_info(self(), :stop) {[DBConnection.execute!(conn, query, [demand])], query} end stop = fn(conn, _, query) -> DBConnection.close!(conn, query) end @@ -58,22 +58,20 @@ defmodule StageTest do {:ok, pool} = P.start_link(opts) start = &DBConnection.prepare!(&1, %Q{}) handle = fn(conn, demand, query) -> - GenStage.async_notify(self(), {:producer, :done}) + GenStage.async_info(self(), :stop) {[DBConnection.execute!(conn, query, [demand])], query} end stop = fn(conn, _, query) -> DBConnection.close!(conn, query) end {:ok, stage} = P.producer(pool, start, handle, stop, opts) ref = make_ref() - sub_opts = [cancel: :temporary] + sub_opts = [cancel: :transient] send(stage, {:"$gen_producer", {parent, ref}, {:subscribe, nil, sub_opts}}) sub = {stage, ref} GenStage.ask(sub, 1) assert_receive {:"$gen_consumer", ^sub, [%R{}]} mon = Process.monitor(stage) - GenStage.cancel(sub, :normal) assert_receive {:DOWN, ^mon, :process, ^stage, :normal} - assert_received {:"$gen_consumer", ^sub, {:cancel, :normal}} assert [ connect: [_], @@ -103,7 +101,7 @@ defmodule StageTest do stop = fn(conn, _, query) -> DBConnection.close!(conn, query) end {:ok, stage} = P.producer(pool, start, handle, stop, opts) mon = Process.monitor(stage) - catch_exit(stage |> Flow.from_stage() |> Enum.to_list()) + assert stage |> Flow.from_stage() |> Enum.to_list() == [] assert_receive {:DOWN, ^mon, :process, ^stage, :normal} @@ -135,10 +133,8 @@ defmodule StageTest do end stop = fn(conn, _, query) -> DBConnection.close!(conn, query) end {:ok, stage} = P.producer(pool, start, handle, stop, opts) - mon = Process.monitor(stage) assert stage |> Flow.from_stage() |> Enum.take(1) == [:oops] - - assert_receive {:DOWN, ^mon, :process, ^stage, :normal} + GenStage.stop(stage) assert [ connect: [_], @@ -236,21 +232,22 @@ defmodule StageTest do stop = fn(conn, _, query) -> DBConnection.close!(conn, query) end {:ok, stage} = P.consumer(pool, start, handle, stop, opts) - :ok = GenStage.async_subscribe(stage, [to: self(), cancel: :temporary, max_demand: 1]) + :ok = GenStage.async_subscribe(stage, [to: self(), cancel: :transient, max_demand: 1]) assert_receive {:"$gen_producer", {^stage, ref1} = sub1, {:subscribe, _, _}} assert_receive {:"$gen_producer", ^sub1, {:ask, 1}} + :sys.suspend(stage) send(stage, {:"$gen_consumer", {self(), ref1}, [:param]}) - send(stage, {{self(), ref1}, {:producer, :done}}) - :ok = GenStage.async_subscribe(stage, [to: self(), cancel: :temporary, max_demand: 1]) + send(stage, {:"$gen_consumer", {self(), ref1}, {:cancel, :normal}}) + :ok = GenStage.async_subscribe(stage, [to: self(), cancel: :transient, max_demand: 1]) + mon = Process.monitor(stage) + :sys.resume(stage) assert_receive {:"$gen_producer", {^stage, _} = sub2, {:subscribe, _, _}} assert_receive {:"$gen_producer", ^sub1, {:ask, 1}} - assert_receive {:"$gen_producer", ^sub1, {:cancel, :normal}} assert_receive {:"$gen_producer", ^sub2, {:cancel, :normal}} - - GenStage.stop(stage) + assert_receive {:DOWN, ^mon, :process, ^stage, :normal} refute_received {:"$gen_producer", _, _} assert [ @@ -281,10 +278,16 @@ defmodule StageTest do {[DBConnection.execute!(conn, query, events)], query} end stop = fn(conn, _, query) -> DBConnection.close!(conn, query) end + {:ok, stage} = P.producer_consumer(pool, start, handle, stop, opts) mon = Process.monitor(stage) - {:ok, _} = Flow.from_enumerable([:param]) |> Flow.into_stages([stage]) - assert Flow.from_stage(stage) |> Enum.to_list() == [%R{}] + {:ok, flow} = + [:param] + |> Flow.from_enumerable() + #|> Flow.map(&(&1)) + |> Flow.into_stages([stage], [demand: :accumulate]) + + assert GenStage.stream([{flow, [cancel: :transient]}]) |> Enum.to_list() == [%R{}] assert_receive {:DOWN, ^mon, :process, ^stage, :normal} @@ -319,7 +322,7 @@ defmodule StageTest do stop = fn(conn, _, query) -> DBConnection.close!(conn, query) end {:ok, stage} = P.producer_consumer(pool, start, handle, stop, opts) - :ok = GenStage.async_subscribe(stage, [to: self(), cancel: :temporary, max_demand: 1]) + :ok = GenStage.async_subscribe(stage, [to: self(), cancel: :transient, max_demand: 1]) assert_receive {:"$gen_producer", {^stage, ref1} = sub1, {:subscribe, _, _}} assert_receive {:"$gen_producer", ^sub1, {:ask, 1}} @@ -332,15 +335,16 @@ defmodule StageTest do assert_receive %R{} :sys.suspend(stage) - send(stage, {{self(), ref1}, {:producer, :done}}) - :ok = GenStage.async_subscribe(stage, [to: self(), cancel: :temporary, max_demand: 1]) + send(stage, {:"$gen_consumer", {self(), ref1}, {:cancel, :normal}}) + :ok = GenStage.async_subscribe(stage, [to: self(), cancel: :transient, max_demand: 1]) mon = Process.monitor(stage) :sys.resume(stage) assert Task.await(task) == [%R{}] - assert_receive {:"$gen_producer", {^stage, _}, {:subscribe, _, _}} + assert_receive {:"$gen_producer", {^stage, _} = sub2, {:subscribe, _, _}} + assert_receive {:"$gen_producer", ^sub1, {:ask, 1}} + assert_receive {:"$gen_producer", ^sub2, {:cancel, :normal}} assert_receive {:DOWN, ^mon, :process, ^stage, :normal} - assert_received {:"$gen_producer", ^sub1, {:ask, 1}} refute_received {:"$gen_producer", _, _} assert [ @@ -376,26 +380,24 @@ defmodule StageTest do mon = Process.monitor(stage) - {:ok, _} = [:param] |> Flow.from_enumerable() |> Flow.into_stages([stage]) + {:ok, flow} = + [:param] + |> Flow.from_enumerable() + |> Flow.into_stages([stage], [demand: :accumulate]) send(stage, {:"$gen_producer", {self(), mon}, {:subscribe, nil, []}}) send(stage, {:"$gen_producer", {self(), mon}, {:ask, 1000}}) - - sub1 = {stage, mon} - - assert_receive {:"$gen_consumer", ^sub1, [%R{}]} - assert_receive {:"$gen_consumer", ^sub1, {:notification, {:producer, :done}}} - + task = Task.async(fn() -> [stage] |> Flow.from_stages() |> Enum.to_list() end) - :timer.sleep(100) + GenStage.demand(flow, :forward) + sub1 = {stage, mon} + assert_receive {:"$gen_consumer", ^sub1, [%R{}]} GenStage.cancel(sub1, :normal) - assert Task.await(task) == [] - assert_receive {:DOWN, ^mon, _, _, :normal} assert [ @@ -472,7 +474,7 @@ defmodule StageTest do ] = A.record(agent) end - test "stream notifies new consumers that it's done" do + test "stream stops normally after it's done" do stack = [ {:ok, :state}, {:ok, :began, :new_state}, @@ -488,20 +490,15 @@ defmodule StageTest do {:ok, pool} = P.start_link(opts) {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) - ref = make_ref() - send(stage, {:"$gen_producer", {parent, ref}, {:subscribe, nil, []}}) + ref = Process.monitor(stage) + send(stage, {:"$gen_producer", {parent, ref}, {:subscribe, nil, [cancel: :transient]}}) sub = {stage, ref} GenStage.ask(sub, 1000) assert_receive {:"$gen_consumer", ^sub, [%R{}]} - assert_receive {:"$gen_consumer", ^sub, {:notification, {:producer, :halted}}} - assert stage |> Flow.from_stage() |> Enum.to_list() == [] - - mon = Process.monitor(stage) - GenStage.cancel(sub, :normal) - assert_receive {:DOWN, ^mon, :process, ^stage, :normal} - assert_received {:"$gen_consumer", ^sub, {:cancel, :normal}} + assert_receive {:DOWN, ^ref, :process, ^stage, :normal} + refute_received {:"$gen_producer", ^sub, _} assert [ connect: [_], @@ -528,13 +525,13 @@ defmodule StageTest do {:ok, pool} = P.start_link(opts) {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) - ref = make_ref() + ref = Process.monitor(stage) send(stage, {:"$gen_producer", {parent, ref}, {:subscribe, nil, []}}) sub = {stage, ref} - mon = Process.monitor(stage) GenStage.cancel(sub, :normal) - assert_receive {:DOWN, ^mon, :process, ^stage, :normal} + assert_receive {:DOWN, ^ref, :process, ^stage, :normal} + assert_receive {:"$gen_consumer", ^sub, {:cancel, :normal}} assert [ connect: [_], diff --git a/lib/db_connection/stage.ex b/lib/db_connection/stage.ex index cb15087f..1863f7b7 100644 --- a/lib/db_connection/stage.ex +++ b/lib/db_connection/stage.ex @@ -24,7 +24,7 @@ defmodule DBConnection.Stage do @enforce_keys [:conn, :handle, :stop, :state, :opts, :type] defstruct [:conn, :handle, :stop, :state, :opts, :type, - consumers: [], producers: %{}, active: [], done?: false] + producers: %{}, done?: false] @start_opts [:name, :spawn_opt, :debug] @stage_opts [:demand, :buffer_size, :buffer_keep, :dispatcher, :subscribe_to] @@ -279,73 +279,34 @@ defmodule DBConnection.Stage do @doc false def handle_subscribe(:producer, _, {pid, ref} = from, stage) do case stage do - %Stage{done?: true, type: :consumer, producers: producers} -> + %Stage{done?: true} -> GenStage.cancel(from, :normal, [:noconnect]) - stage = %Stage{stage | producers: Map.put(producers, ref, pid)} {:manual, stage} - %Stage{done?: true, type: :producer_consumer, producers: producers} -> - stage = %Stage{stage | producers: Map.put(producers, ref, pid)} - {:manual, stage} - %Stage{done?: false, producers: producers, active: active} -> - stage = %Stage{stage | producers: Map.put(producers, ref, pid), - active: [ref | active]} - {:automatic, stage} + %Stage{done?: false, producers: producers} -> + {:automatic, %Stage{stage | producers: Map.put(producers, ref, pid)}} end end - def handle_subscribe(:consumer, _, {_, ref}, stage) do - %Stage{done?: done?, consumers: consumers} = stage - if done?, do: GenStage.async_notify(self(), {:producer, :done}) - {:automatic, %Stage{stage | consumers: [ref | consumers]}} + def handle_subscribe(:consumer, _, _, stage) do + {:automatic, stage} end @doc false - def handle_cancel(_, {_, ref}, stage) do - %Stage{type: type, consumers: consumers, producers: producers, - active: active, done?: done?} = stage - case producers do - %{^ref => _} when active != [ref] or done? -> - producers = Map.delete(producers, ref) - active = List.delete(active, ref) - {:noreply, [], %Stage{stage | active: active, producers: producers}} - %{^ref => _} when type == :consumer -> - producers = Map.delete(producers, ref) - for {ref, pid} <- producers do - GenStage.cancel({pid, ref}, :normal, [:noconnect]) - end - stage = %Stage{stage | active: [], done?: true, producers: producers} - {:noreply, [], stage} - %{^ref => _} when type == :producer_consumer -> - producers = Map.delete(producers, ref) - GenStage.async_notify(self(), {:producer, :done}) - stage = %Stage{stage | active: [], done?: true, producers: producers} - {:noreply, [], stage} - %{} when consumers == [ref] -> - {:stop, :normal, %Stage{stage | consumers: []}} - %{} -> - consumers = List.delete(consumers, ref) - {:noreply, [], %Stage{stage | consumers: consumers}} + def handle_cancel(_, _, %Stage{done?: true} = stage) do + {:noreply, stage} + end + def handle_cancel(_, {_, ref}, %Stage{done?: false, producers: producers} = stage) do + case Map.delete(producers, ref) do + new_producers when new_producers == %{} and producers != %{} -> + GenStage.async_info(self(), :stop) + {:noreply, [], %Stage{stage | done?: true, producers: %{}}} + new_producers -> + {:noreply, [], %Stage{stage | producers: new_producers}} end end @doc false - def handle_info({{_, ref}, {:producer, state}}, stage) when state in [:halted, :done] do - %Stage{type: type, producers: producers, active: active, - done?: done?} = stage - case producers do - %{^ref => _} when active != [ref] or done? -> - active = List.delete(active, ref) - {:noreply, [], %Stage{stage | active: active}} - %{^ref => _} when type == :consumer -> - for {ref, pid} <- producers do - GenStage.cancel({pid, ref}, :normal, [:noconnect]) - end - {:noreply, [], %Stage{stage | active: [], done?: true}} - %{^ref => _} when type == :producer_consumer -> - GenStage.async_notify(self(), {:producer, :done}) - {:noreply, [], %Stage{stage | active: [], done?: true}} - %{} -> - {:noreply, [], stage} - end + def handle_info(:stop, stage) do + {:stop, :normal, stage} end def handle_info(_, stage) do {:noreply, [], stage} @@ -430,12 +391,12 @@ defmodule DBConnection.Stage do {:suspended, {0, acc}, cont} -> {Enum.reverse(acc), {:cont, cont}} {state, {_, acc}} when state in [:halted, :done] -> - GenStage.async_notify(self(), {:producer, state}) + GenStage.async_info(self(), :stop) {Enum.reverse(acc), state} end end defp stream_next(_, _, state) when state in [:halted, :done] do - GenStage.async_notify(self(), {:producer, state}) + GenStage.async_info(self(), :stop) {[], state} end diff --git a/mix.exs b/mix.exs index 307994e6..3ea9fd58 100644 --- a/mix.exs +++ b/mix.exs @@ -30,8 +30,10 @@ defmodule DBConnection.Mixfile do [{:connection, "~> 1.0.2"}, {:poolboy, "~> 1.5", [optional: true]}, {:sbroker, "~> 1.0", [optional: true]}, - {:gen_stage, "~> 0.11", [optional: true]}, - {:flow, "~> 0.11", [optional: true]}, + {:gen_stage, "~> 0.11", [optional: true, github: "elixir-lang/gen_stage", + branch: "jv-exit-signals"]}, + {:flow, "~> 0.11", [optional: true, github: "elixir-lang/flow", + branch: "jv-exit-signals"]}, {:ex_doc, "~> 0.12", only: :dev}] end diff --git a/mix.lock b/mix.lock index 1c93c1e7..33473b27 100644 --- a/mix.lock +++ b/mix.lock @@ -1,7 +1,7 @@ %{"connection": {:hex, :connection, "1.0.4", "a1cae72211f0eef17705aaededacac3eb30e6625b04a6117c1b2db6ace7d5976", [:mix], []}, "earmark": {:hex, :earmark, "0.2.1", "ba6d26ceb16106d069b289df66751734802777a3cbb6787026dd800ffeb850f3", [:mix], []}, "ex_doc": {:hex, :ex_doc, "0.12.0", "b774aabfede4af31c0301aece12371cbd25995a21bb3d71d66f5c2fe074c603f", [:mix], [{:earmark, "~> 0.2", [hex: :earmark, optional: false]}]}, - "flow": {:hex, :flow, "0.11.1", "cbc35a0236520cc5fec7b5863cd8431cb1e77297c5c9119055676355eb1fb5a6", [:mix], [{:gen_stage, "~> 0.11.0", [hex: :gen_stage, optional: false]}]}, - "gen_stage": {:hex, :gen_stage, "0.11.0", "943bdfa85c75fa624e0a36a9d135baad20a523be040178f5a215444b45c66ea4", [:mix], []}, + "flow": {:git, "https://github.com/elixir-lang/flow.git", "cfdc0d240aa990c5229c70e032595f2b063a1445", [branch: "jv-exit-signals"]}, + "gen_stage": {:git, "https://github.com/elixir-lang/gen_stage.git", "966d998c699dec58e0d183ddbaa2b783c46cd375", [branch: "jv-exit-signals"]}, "poolboy": {:hex, :poolboy, "1.5.1", "6b46163901cfd0a1b43d692657ed9d7e599853b3b21b95ae5ae0a777cf9b6ca8", [:rebar], []}, "sbroker": {:hex, :sbroker, "1.0.0", "28ff1b5e58887c5098539f236307b36fe1d3edaa2acff9d6a3d17c2dcafebbd0", [:rebar3], []}} From 190e03274c4ad2c061a101ee12fba19369b7573f Mon Sep 17 00:00:00 2001 From: James Fish Date: Sat, 27 May 2017 17:16:18 +0100 Subject: [PATCH 09/20] Only support stage that streams query --- integration_test/cases/stage_test.exs | 453 +------------------------- lib/db_connection.ex | 53 ++- lib/db_connection/stage.ex | 395 ++++------------------ test/test_support.exs | 22 +- 4 files changed, 116 insertions(+), 807 deletions(-) diff --git a/integration_test/cases/stage_test.exs b/integration_test/cases/stage_test.exs index 5cb529ef..d22467c0 100644 --- a/integration_test/cases/stage_test.exs +++ b/integration_test/cases/stage_test.exs @@ -7,410 +7,7 @@ defmodule StageTest do alias TestCursor, as: C alias TestResult, as: R - test "producer executes on demand" do - stack = [ - {:ok, :state}, - {:ok, :began, :new_state}, - {:ok, %Q{}, :newer_state}, - {:ok, %R{}, :newest_state}, - {:ok, :closed, :state2}, - {:ok, :commited, :new_state2} - ] - {:ok, agent} = A.start_link(stack) - - opts = [agent: agent, parent: self()] - {:ok, pool} = P.start_link(opts) - start = &DBConnection.prepare!(&1, %Q{}) - handle = fn(conn, demand, query) -> - GenStage.async_info(self(), :stop) - {[DBConnection.execute!(conn, query, [demand])], query} - end - stop = fn(conn, _, query) -> DBConnection.close!(conn, query) end - {:ok, stage} = P.producer(pool, start, handle, stop, opts) - mon = Process.monitor(stage) - assert stage |> Flow.from_stage() |> Enum.to_list() == [%R{}] - - assert_receive {:DOWN, ^mon, :process, ^stage, :normal} - - assert [ - connect: [_], - handle_begin: [_, :state], - handle_prepare: [%Q{}, _, :new_state], - handle_execute: [%Q{}, _, _, :newer_state], - handle_close: [%Q{}, _, :newest_state], - handle_commit: [_, :state2] - ] = A.record(agent) - end - - test "producer exits when last consumer cancels" do - stack = [ - {:ok, :state}, - {:ok, :began, :new_state}, - {:ok, %Q{}, :newer_state}, - {:ok, %R{}, :newest_state}, - {:ok, :closed, :state2}, - {:ok, :commited, :new_state2} - ] - {:ok, agent} = A.start_link(stack) - - parent = self() - opts = [agent: agent, parent: parent] - {:ok, pool} = P.start_link(opts) - start = &DBConnection.prepare!(&1, %Q{}) - handle = fn(conn, demand, query) -> - GenStage.async_info(self(), :stop) - {[DBConnection.execute!(conn, query, [demand])], query} - end - stop = fn(conn, _, query) -> DBConnection.close!(conn, query) end - {:ok, stage} = P.producer(pool, start, handle, stop, opts) - - ref = make_ref() - sub_opts = [cancel: :transient] - send(stage, {:"$gen_producer", {parent, ref}, {:subscribe, nil, sub_opts}}) - sub = {stage, ref} - GenStage.ask(sub, 1) - assert_receive {:"$gen_consumer", ^sub, [%R{}]} - mon = Process.monitor(stage) - assert_receive {:DOWN, ^mon, :process, ^stage, :normal} - - assert [ - connect: [_], - handle_begin: [_, :state], - handle_prepare: [%Q{}, _, :new_state], - handle_execute: [%Q{}, [1], _, :newer_state], - handle_close: [%Q{}, _, :newest_state], - handle_commit: [_, :state2] - ] = A.record(agent) - end - - test "producer does not stop or commit on rollback" do - stack = [ - {:ok, :state}, - {:ok, :began, :new_state}, - {:ok, %Q{}, :newer_state}, - {:ok, :rolledback, :newest_state} - ] - {:ok, agent} = A.start_link(stack) - - opts = [agent: agent, parent: self()] - {:ok, pool} = P.start_link(opts) - start = &DBConnection.prepare!(&1, %Q{}) - handle = fn(conn, _, _) -> - DBConnection.rollback(conn, :normal) - end - stop = fn(conn, _, query) -> DBConnection.close!(conn, query) end - {:ok, stage} = P.producer(pool, start, handle, stop, opts) - mon = Process.monitor(stage) - assert stage |> Flow.from_stage() |> Enum.to_list() == [] - - assert_receive {:DOWN, ^mon, :process, ^stage, :normal} - - assert [ - connect: [_], - handle_begin: [_, :state], - handle_prepare: [%Q{}, _, :new_state], - handle_rollback: [_, :newer_state] - ] = A.record(agent) - end - - test "producer does not stop or commit on handle nested rollback" do - stack = [ - {:ok, :state}, - {:ok, :began, :new_state}, - {:ok, %Q{}, :newer_state}, - {:ok, :rolledback, :newest_state} - ] - {:ok, agent} = A.start_link(stack) - - opts = [agent: agent, parent: self()] - {:ok, pool} = P.start_link(opts) - start = &DBConnection.prepare!(&1, %Q{}) - handle = fn(conn, _, query) -> - assert DBConnection.transaction(conn, fn(conn2) -> - DBConnection.rollback(conn2, :oops) - end) == {:error, :oops} - {[:oops], query} - end - stop = fn(conn, _, query) -> DBConnection.close!(conn, query) end - {:ok, stage} = P.producer(pool, start, handle, stop, opts) - assert stage |> Flow.from_stage() |> Enum.take(1) == [:oops] - GenStage.stop(stage) - - assert [ - connect: [_], - handle_begin: [_, :state], - handle_prepare: [%Q{}, _, :new_state], - handle_rollback: [_, :newer_state] - ] = A.record(agent) - end - - test "producer exits but does not commit on start nested rollback" do - stack = [ - {:ok, :state}, - {:ok, :began, :new_state}, - {:ok, :rolledback, :newer_state} - ] - {:ok, agent} = A.start_link(stack) - - opts = [agent: agent, parent: self()] - {:ok, pool} = P.start_link(opts) - start = fn(conn) -> - assert DBConnection.transaction(conn, fn(conn2) -> - DBConnection.rollback(conn2, :oops) - end) == {:error, :oops} - :oops - end - fail = fn(_, _, _) -> flunk "should not run" end - {:ok, stage} = P.producer(pool, start, fail, fail, opts) - - Process.flag(:trap_exit, true) - catch_exit(stage |> Flow.from_stage() |> Enum.to_list()) - assert_receive {:EXIT, ^stage, :rollback} - - assert [ - connect: [_], - handle_begin: [_, :state], - handle_rollback: [_, :new_state] - ] = A.record(agent) - end - - test "consumer executes on events" do - stack = [ - {:ok, :state}, - {:ok, :began, :new_state}, - {:ok, %Q{}, :newer_state}, - {:ok, %R{}, :newest_state}, - {:ok, :closed, :state2}, - {:ok, :commited, :new_state2} - ] - {:ok, agent} = A.start_link(stack) - - opts = [agent: agent, parent: self()] - {:ok, pool} = P.start_link(opts) - start = &DBConnection.prepare!(&1, %Q{}) - handle = fn(conn, events, query) -> - DBConnection.execute!(conn, query, events) - {[], query} - end - stop = fn(conn, _, query) -> DBConnection.close!(conn, query) end - {:ok, stage} = P.consumer(pool, start, handle, stop, opts) - mon = Process.monitor(stage) - flow = Flow.from_enumerable([:param]) - {:ok, _} = Flow.into_stages(flow, [stage]) - - assert_receive {:DOWN, ^mon, :process, ^stage, :normal} - - assert [ - connect: [_], - handle_begin: [_, :state], - handle_prepare: [%Q{}, _, :new_state], - handle_execute: [%Q{}, [:param], _, :newer_state], - handle_close: [%Q{}, _, :newest_state], - handle_commit: [_, :state2] - ] = A.record(agent) - end - - test "consumer cancels new producers after it's done" do - stack = [ - {:ok, :state}, - {:ok, :began, :new_state}, - {:ok, %Q{}, :newer_state}, - {:ok, %R{}, :newest_state}, - {:ok, :closed, :state2}, - {:ok, :commited, :new_state2} - ] - {:ok, agent} = A.start_link(stack) - - parent = self() - opts = [agent: agent, parent: parent] - {:ok, pool} = P.start_link(opts) - start = &DBConnection.prepare!(&1, %Q{}) - handle = fn(conn, events, query) -> - DBConnection.execute!(conn, query, events) - {[], query} - end - stop = fn(conn, _, query) -> DBConnection.close!(conn, query) end - {:ok, stage} = P.consumer(pool, start, handle, stop, opts) - - :ok = GenStage.async_subscribe(stage, [to: self(), cancel: :transient, max_demand: 1]) - - assert_receive {:"$gen_producer", {^stage, ref1} = sub1, {:subscribe, _, _}} - assert_receive {:"$gen_producer", ^sub1, {:ask, 1}} - - :sys.suspend(stage) - send(stage, {:"$gen_consumer", {self(), ref1}, [:param]}) - send(stage, {:"$gen_consumer", {self(), ref1}, {:cancel, :normal}}) - :ok = GenStage.async_subscribe(stage, [to: self(), cancel: :transient, max_demand: 1]) - mon = Process.monitor(stage) - :sys.resume(stage) - - assert_receive {:"$gen_producer", {^stage, _} = sub2, {:subscribe, _, _}} - assert_receive {:"$gen_producer", ^sub1, {:ask, 1}} - assert_receive {:"$gen_producer", ^sub2, {:cancel, :normal}} - assert_receive {:DOWN, ^mon, :process, ^stage, :normal} - refute_received {:"$gen_producer", _, _} - - assert [ - connect: [_], - handle_begin: [_, :state], - handle_prepare: [%Q{}, _, :new_state], - handle_execute: [%Q{}, [:param], _, :newer_state], - handle_close: [%Q{}, _, :newest_state], - handle_commit: [_, :state2] - ] = A.record(agent) - end - - test "producer_consumer executes on events" do - stack = [ - {:ok, :state}, - {:ok, :began, :new_state}, - {:ok, %Q{}, :newer_state}, - {:ok, %R{}, :newest_state}, - {:ok, :closed, :state2}, - {:ok, :commited, :new_state2} - ] - {:ok, agent} = A.start_link(stack) - - opts = [agent: agent, parent: self()] - {:ok, pool} = P.start_link(opts) - start = &DBConnection.prepare!(&1, %Q{}) - handle = fn(conn, events, query) -> - {[DBConnection.execute!(conn, query, events)], query} - end - stop = fn(conn, _, query) -> DBConnection.close!(conn, query) end - - {:ok, stage} = P.producer_consumer(pool, start, handle, stop, opts) - mon = Process.monitor(stage) - {:ok, flow} = - [:param] - |> Flow.from_enumerable() - #|> Flow.map(&(&1)) - |> Flow.into_stages([stage], [demand: :accumulate]) - - assert GenStage.stream([{flow, [cancel: :transient]}]) |> Enum.to_list() == [%R{}] - - assert_receive {:DOWN, ^mon, :process, ^stage, :normal} - - assert [ - connect: [_], - handle_begin: [_, :state], - handle_prepare: [%Q{}, _, :new_state], - handle_execute: [%Q{}, [:param], _, :newer_state], - handle_close: [%Q{}, _, :newest_state], - handle_commit: [_, :state2] - ] = A.record(agent) - end - - test "producer_consumer does not send demand to new producers after done" do - stack = [ - {:ok, :state}, - {:ok, :began, :new_state}, - {:ok, %Q{}, :newer_state}, - {:ok, %R{}, :newest_state}, - {:ok, :closed, :state2}, - {:ok, :commited, :new_state2} - ] - {:ok, agent} = A.start_link(stack) - - parent = self() - opts = [agent: agent, parent: parent] - {:ok, pool} = P.start_link(opts) - start = &DBConnection.prepare!(&1, %Q{}) - handle = fn(conn, events, query) -> - {[DBConnection.execute!(conn, query, events)], query} - end - stop = fn(conn, _, query) -> DBConnection.close!(conn, query) end - {:ok, stage} = P.producer_consumer(pool, start, handle, stop, opts) - - :ok = GenStage.async_subscribe(stage, [to: self(), cancel: :transient, max_demand: 1]) - - assert_receive {:"$gen_producer", {^stage, ref1} = sub1, {:subscribe, _, _}} - assert_receive {:"$gen_producer", ^sub1, {:ask, 1}} - - task = Task.async(fn() -> - [stage] |> Flow.from_stages() |> Enum.map(&send(parent, &1)) - end) - - send(stage, {:"$gen_consumer", {self(), ref1}, [:param]}) - assert_receive %R{} - - :sys.suspend(stage) - send(stage, {:"$gen_consumer", {self(), ref1}, {:cancel, :normal}}) - :ok = GenStage.async_subscribe(stage, [to: self(), cancel: :transient, max_demand: 1]) - mon = Process.monitor(stage) - :sys.resume(stage) - - assert Task.await(task) == [%R{}] - assert_receive {:"$gen_producer", {^stage, _} = sub2, {:subscribe, _, _}} - assert_receive {:"$gen_producer", ^sub1, {:ask, 1}} - assert_receive {:"$gen_producer", ^sub2, {:cancel, :normal}} - assert_receive {:DOWN, ^mon, :process, ^stage, :normal} - refute_received {:"$gen_producer", _, _} - - assert [ - connect: [_], - handle_begin: [_, :state], - handle_prepare: [%Q{}, _, :new_state], - handle_execute: [%Q{}, [:param], _, :newer_state], - handle_close: [%Q{}, _, :newest_state], - handle_commit: [_, :state2] - ] = A.record(agent) - end - - test "producer_consumer does notifies new consumers after done" do - stack = [ - {:ok, :state}, - {:ok, :began, :new_state}, - {:ok, %Q{}, :newer_state}, - {:ok, %R{}, :newest_state}, - {:ok, :closed, :state2}, - {:ok, :commited, :new_state2} - ] - {:ok, agent} = A.start_link(stack) - - parent = self() - opts = [agent: agent, parent: parent] - {:ok, pool} = P.start_link(opts) - start = &DBConnection.prepare!(&1, %Q{}) - handle = fn(conn, events, query) -> - {[DBConnection.execute!(conn, query, events)], query} - end - stop = fn(conn, _, query) -> DBConnection.close!(conn, query) end - {:ok, stage} = P.producer_consumer(pool, start, handle, stop, opts) - - mon = Process.monitor(stage) - - {:ok, flow} = - [:param] - |> Flow.from_enumerable() - |> Flow.into_stages([stage], [demand: :accumulate]) - - send(stage, {:"$gen_producer", {self(), mon}, {:subscribe, nil, []}}) - send(stage, {:"$gen_producer", {self(), mon}, {:ask, 1000}}) - - task = Task.async(fn() -> - [stage] |> Flow.from_stages() |> Enum.to_list() - end) - - GenStage.demand(flow, :forward) - sub1 = {stage, mon} - assert_receive {:"$gen_consumer", ^sub1, [%R{}]} - - GenStage.cancel(sub1, :normal) - assert Task.await(task) == [] - assert_receive {:DOWN, ^mon, _, _, :normal} - - assert [ - connect: [_], - handle_begin: [_, :state], - handle_prepare: [%Q{}, _, :new_state], - handle_execute: [%Q{}, [:param], _, :newer_state], - handle_close: [%Q{}, _, :newest_state], - handle_commit: [_, :state2] - ] = A.record(agent) - end - - test "stream returns result" do + test "start_link with GenStage.stream returns result" do stack = [ {:ok, :state}, {:ok, :began, :new_state}, @@ -426,7 +23,7 @@ defmodule StageTest do {:ok, pool} = P.start_link(opts) {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) mon = Process.monitor(stage) - assert stage |> Flow.from_stage() |> Enum.to_list() == [%R{}, %R{}] + assert [stage] |> GenStage.stream([cancel: :transient]) |> Enum.to_list() == [%R{}, %R{}] assert_receive {:DOWN, ^mon, :process, ^stage, :normal} @@ -441,7 +38,7 @@ defmodule StageTest do ] = A.record(agent) end - test "prepare_stream returns result" do + test "start_link with prepare: true returns result" do stack = [ {:ok, :state}, {:ok, :began, :new_state}, @@ -456,9 +53,9 @@ defmodule StageTest do opts = [agent: agent, parent: self()] {:ok, pool} = P.start_link(opts) - {:ok, stage} = P.prepare_stream_stage(pool, %Q{}, [:param], opts) + {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], [prepare: true] ++ opts) mon = Process.monitor(stage) - assert stage |> Flow.from_stage() |> Enum.to_list() == [%R{}, %R{}] + assert [stage] |> GenStage.stream([cancel: :transient]) |> Enum.to_list() == [%R{}, %R{}] assert_receive {:DOWN, ^mon, :process, ^stage, :normal} @@ -510,38 +107,6 @@ defmodule StageTest do ] = A.record(agent) end - test "stream finishes when consumers do" do - stack = [ - {:ok, :state}, - {:ok, :began, :new_state}, - {:ok, %C{}, :newer_state}, - {:ok, :deallocated, :newest_state}, - {:ok, :commited, :state2} - ] - {:ok, agent} = A.start_link(stack) - - parent = self() - opts = [agent: agent, parent: parent] - {:ok, pool} = P.start_link(opts) - {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) - - ref = Process.monitor(stage) - send(stage, {:"$gen_producer", {parent, ref}, {:subscribe, nil, []}}) - sub = {stage, ref} - - GenStage.cancel(sub, :normal) - assert_receive {:DOWN, ^ref, :process, ^stage, :normal} - assert_receive {:"$gen_consumer", ^sub, {:cancel, :normal}} - - assert [ - connect: [_], - handle_begin: [_, :state], - handle_declare: [%Q{}, [:param], _, :new_state], - handle_deallocate: [%Q{}, %C{}, _, :newer_state], - handle_commit: [_, :newest_state] - ] = A.record(agent) - end - test "stream rolls back on abnormal exit" do stack = [ {:ok, :state}, @@ -655,7 +220,7 @@ defmodule StageTest do {:ok, pool} = P.start_link(opts) Process.flag(:trap_exit, true) {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) - catch_exit(stage |> Flow.from_stage() |> Enum.to_list()) + catch_exit([stage] |> GenStage.stream([cancel: :transient]) |> Enum.to_list()) assert_receive {:EXIT, ^stage, {^err, _}} assert [ @@ -688,7 +253,7 @@ defmodule StageTest do {:ok, pool} = P.start_link(opts) Process.flag(:trap_exit, true) {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) - catch_exit(stage |> Flow.from_stage() |> Enum.to_list()) + catch_exit([stage] |> GenStage.stream([cancel: :transient]) |> Enum.to_list()) assert_receive {:EXIT, ^stage, {^err, _}} assert_receive :reconnected @@ -720,7 +285,7 @@ defmodule StageTest do {:ok, pool} = P.start_link(opts) Process.flag(:trap_exit, true) {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) - catch_exit(stage |> Flow.from_stage() |> Enum.to_list()) + catch_exit([stage] |> GenStage.stream([cancel: :transient]) |> Enum.to_list()) assert_receive {:EXIT, ^stage, {^err, _}} assert [ @@ -754,7 +319,7 @@ defmodule StageTest do {:ok, pool} = P.start_link(opts) Process.flag(:trap_exit, true) {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) - catch_exit(stage |> Flow.from_stage() |> Enum.to_list()) + catch_exit([stage] |> GenStage.stream([cancel: :transient]) |> Enum.to_list()) assert_receive {:EXIT, ^stage, {^err, _}} assert_receive :reconnected diff --git a/lib/db_connection.ex b/lib/db_connection.ex index 5feac0e9..21b8c247 100644 --- a/lib/db_connection.ex +++ b/lib/db_connection.ex @@ -834,6 +834,9 @@ defmodule DBConnection do ### Options + * `:stream_map` - A function to flat map the results of the query, either a + 2-arity fun, `{module, function, args}` with `DBConnection.t` and the result + prepended to `args` or `nil` (default: `nil` * `:pool_timeout` - The maximum time to wait for a reply when making a synchronous call to the pool (default: `5_000`) * `:queue` - Whether to block waiting in an internal queue for the @@ -870,6 +873,9 @@ defmodule DBConnection do ### Options + * `:stream_map` - A function to flat map the results of the query, either a + 2-arity fun, `{module, function, args}` with `DBConnection.t` and the result + prepended to `args` or `nil` (default: `nil`) * `:pool_timeout` - The maximum time to wait for a reply when making a synchronous call to the pool (default: `5_000`) * `:queue` - Whether to block waiting in an internal queue for the @@ -890,7 +896,8 @@ defmodule DBConnection do {:ok, results} = DBConnection.transaction(conn, fn(conn) -> query = %Query{statement: "SELECT id FROM table"} query = DBConnection.prepare!(conn, query) - stream = DBConnection.stream(conn, query, []) + opts = [stream_map: &Map.fetch!(&1, :rows)] + stream = DBConnection.stream(conn, query, [], opts) Enum.to_list(stream) end) """ @@ -1462,7 +1469,8 @@ defmodule DBConnection do end end - defp prepare_declare(conn, query, params, opts) do + @doc false + def prepare_declare(conn, query, params, opts) do query = parse(:prepare_declare, query, params, opts) case run_prepare_declare(conn, query, params, opts) do {{:ok, query, cursor}, meter} -> @@ -1498,7 +1506,8 @@ defmodule DBConnection do end end - defp declare(conn, query, params, opts) do + @doc false + def declare(conn, query, params, opts) do encoded = encode(:declare, query, params, opts) case run_declare(conn, query, encoded, opts) do {{:ok, cursor}, meter} -> @@ -1527,17 +1536,27 @@ defmodule DBConnection do end end - defp fetch(conn, {:first, query, cursor}, opts) do - fetch(conn, :handle_first, :first, query, cursor, opts) + @doc false + def fetch(conn, state, opts) do + case run_fetch(conn, state, opts) do + {:ok, result, state} -> + {fetch_map(conn, result, opts), state} + {:halt, _} = halt -> + halt + end + end + + defp run_fetch(conn, {:first, query, cursor}, opts) do + run_fetch(conn, :handle_first, :first, query, cursor, opts) end - defp fetch(conn, {:next, query, cursor}, opts) do - fetch(conn, :handle_next, :next, query, cursor, opts) + defp run_fetch(conn, {:next, query, cursor}, opts) do + run_fetch(conn, :handle_next, :next, query, cursor, opts) end - defp fetch(_, {:deallocate, _, _} = state, _) do + defp run_fetch(_, {:deallocate, _, _} = state, _) do {:halt, state} end - def fetch(conn, fun, call, query, cursor, opts) do + def run_fetch(conn, fun, call, query, cursor, opts) do fetch = &handle(&1, fun, [query, cursor], opts) case run_meter(conn, fetch, opts) do {{:ok, result}, meter} -> @@ -1552,10 +1571,22 @@ defmodule DBConnection do defp fetch_decode(status, call, query, cursor, meter, result, opts) do {:ok, decoded} = decode(call, query, cursor, meter, result, opts) - {[decoded], {status, query, cursor}} + {:ok, decoded, {status, query, cursor}} end - defp deallocate(conn, {_, query, cursor}, opts) do + defp fetch_map(conn, result, opts) do + case Keyword.get(opts, :stream_map) do + map when is_function(map, 2) -> + map.(conn, result) + {mod, fun, args} -> + apply(mod, fun, [conn, result | args]) + nil -> + [result] + end + end + + @doc false + def deallocate(conn, {_, query, cursor}, opts) do case get_info(conn) do :closed -> :ok _ -> deallocate(conn, query, cursor, opts) diff --git a/lib/db_connection/stage.ex b/lib/db_connection/stage.ex index 1863f7b7..6e3faf0d 100644 --- a/lib/db_connection/stage.ex +++ b/lib/db_connection/stage.ex @@ -1,9 +1,15 @@ defmodule DBConnection.Stage do @moduledoc """ - A `GenStage` process that encapsulates a transaction. + A `GenStage` producer that encapsulates a transaction and streams the + result of a query. ### Options + * `:stream_map` - A function to flat map the results of the query, either a + 2-arity fun, `{module, function, args}` with `DBConnection.t` and the result + prepended to `args` or `nil` (default: `nil`) + * `:prepare` - Whether the consumer should prepare the query before + streaming it (default: `false`) * `:pool_timeout` - The maximum time to wait for a reply when making a synchronous call to the pool (default: `5_000`) * `:queue` - Whether to block waiting in an internal queue for the @@ -12,326 +18,84 @@ defmodule DBConnection.Stage do to hold the connection's state (ignored when using a run/transaction connection, default: `15_000`) * `:log` - A function to log information about a call, either - a 1-arity fun, `{module, function, args}` with `DBConnection.LogEntry.t` + a 1-arity fun or `{module, function, args}` with `DBConnection.LogEntry.t` prepended to `args` or `nil`. See `DBConnection.LogEntry` (default: `nil`) The pool and connection module may support other options. All options - are passed to `handle_begin/2`, `handle_commit/2` and `handle_rollback/2`. + are passed to `handle_begin/2`, `handle_prepare/3`, `handle_declare/4`, + `handle_first/4`, `handle_next/4`, `handle_deallocate/4`, `handle_commit/2` + and `handle_rollback/2`. In addition, the demand will be passed to + `handle_first/4` and `handle_next/4` by adding `fetch: demand` to the options. """ alias __MODULE__, as: Stage use GenStage - @enforce_keys [:conn, :handle, :stop, :state, :opts, :type] - defstruct [:conn, :handle, :stop, :state, :opts, :type, - producers: %{}, done?: false] + @enforce_keys [:conn, :state, :opts] + defstruct [:conn, :state, :opts] @start_opts [:name, :spawn_opt, :debug] - @stage_opts [:demand, :buffer_size, :buffer_keep, :dispatcher, :subscribe_to] + @stage_opts [:demand, :buffer_size, :buffer_keep, :dispatcher] @doc """ - Start link a `GenStage` producer that will prepare a query, execute it and - stream results using a cursor inside a transaction. + Start link a `GenStage` producer that will optionally prepare a query, execute + it and stream results using a cursor inside a transaction. The transaction is rolled back if the process terminates with a reason other than `:normal`. - All options are passed to `DBConnection.prepare_stream/4`. For options see - "Options" in the module documentation. - - ### Example - - query = %Query{statement: "SELECT id FROM table"} - {:ok, stage} = DBConnection.Stage.prepare_stream(pool, query, []) - stage |> Flow.from_stage() |> Enum.to_list() - end) - """ - def prepare_stream(pool, query, params, opts \\ []) do - stream(pool, &DBConnection.prepare_stream/4, query, params, opts) - end - - @doc """ - Start link a `GenStage` producer that will execute a query and stream results - using a cursor inside a transaction. - - The transaction is rolled back if the process terminates with a reason other - than `:normal`. - - All options are passed to `DBConnection.stream/4`. For options see "Options" - in the module documentation - - ### Example - - query = DBConnection.prepare!(pool, %Query{statement: "SELECT id FROM table"}) - {:ok, stage} = DBConnection.Stage.stream(pool, query, []) - stage |> Flow.from_stage() |> Enum.to_list() - end) - """ - def stream(pool, query, params, opts \\ []) do - stream(pool, &DBConnection.stream/4, query, params, opts) - end - - @doc """ - Start link a `GenStage` producer that will run a transaction for its duration. - - The first argument is the pool, the second argument is the start function, - the third argument is the handle demand function, the fourth argument is the - stop function and the optional fiftth argument are the options. - - The start function is a 1-arity anonymous function with argument - `DBConnection.t`. This is called after the transaction begins but before - `producer/5` returns. It should return the accumulator or call - `DBConnection.rollback/2` to stop the `GenStage`. - - The handle demand function is a 3-arity anonymous function. The first argument - is the `DBConnection.t` for the transaction, the second argument is the - `demand`, and the third argument is the accumulator. This function returns a - 2-tuple, with first element as list of events to fulfil the demand and second - element as the accumulator. If the producer has emitted all events (and so - not fulfilled demand) it should call - `GenStage.async_notify(self(), {:producer, :done | :halted}` to signal to - consumers that it has finished. Also this function can rollback and stop the - `GenStage` using `DBConnection.rollback/2`. - - The stop function is a 3-arity anonymous function. The first argument is the - `DBConnection.t` for the transaction, the second argument is the terminate - reason and the third argument is the accumulator. This function will only be - called if connection is alive and the transaction has not been rolled back. If - this function returns the transaction is committed. This function can rollback - and stop the `GenStage` using `DBConnection.rollback/2`. - - For options see "Options" in the module documentation. - - The `GenStage` process will behave like a `Flow` stage: - - * It will stop with reason `:normal` when the last consumer cancels - - ### Example - - start = - fn(conn) -> - {ids, DBConnection.prepare!(conn, query, opts)} - end - handle_demand = - fn(conn, demand, {ids, query}) -> - {param, rem} = Enum.split(ids, demand) - %{rows: rows} = DBConection.execute!(conn, query, [param]) - if rem == [], do: GenStage.async_notify(self(), {:producer, :done}) - {rows, {rem, query}} - end - stop = - fn(conn, reason, {rem, query}) -> - DBConnection.close!(conn, query, opts) - if reason != :normal or rem != [] do - DBConnection.rollback(conn, reason) - end - end - DBConnection.Stage.producer(pool, start, handle_demand, stop) - """ - @spec producer(GenServer.server, ((DBConnection.t) -> acc), - ((DBConnection.t, demand :: pos_integer, acc) -> {[any], acc}), - ((DBConnection.t, reason :: any, acc) -> any), Keyword.t) :: - GenServer.on_start when acc: var - def producer(pool, start, handle_demand, stop, opts \\ []) do - start_link(pool, :producer, start, handle_demand, stop, opts) - end - - @doc """ - Start link a `GenStage` producer consumer that will run a transaction for its - duration. - - The first argument is the pool, the second argument is the start function, - the third argument is the handle events function, the fourth argument is the - stop function and the optional fiftth argument are the options. - - The start function is a 1-arity anonymous function with argument - `DBConnection.t`. This is called after the transaction begins but before - `producer_consumer/5` returns. It should return the accumulator or call - `DBConnection.rollback/2` to stop the `GenStage`. - - The handle events function is a 3-arity anonymous function. The first argument - is the `DBConnection.t` for the transaction, the second argument is a list of - incoming events, and the third argument is the accumulator. This function - returns a 2-tuple, with first element as list of outgoing events and second - element as the accumulator. Also this function can rollback and stop the - `GenStage` using `DBConnection.rollback/2`. - - The stop function is a 3-arity anonymous function. The first argument is the - `DBConnection.t` for the transaction, the second argument is the terminate - reason and the third argument is the accumulator. This function will only be - called if connection is alive and the transaction has not been rolled back. If - this function returns the transaction is committed. This function can rollback - and stop the `GenStage` using `DBConnection.rollback/2`. - - For options see "Options" in the module documentation. - - The `GenStage` process will behave like a `Flow` stage: - - * It will stop with reason `:normal` when the last consumer cancels - * It will notify consumers that it is done when all producers have cancelled - or notified that they are done or halted - * It will not send demand to new producers when all producers have notified - that they are done or halted - - ### Example - - start = - fn(conn) -> - DBConnection.prepare!(conn, query, opts) - end - handle_events = - fn(conn, ids, query) -> - %{rows: rows} = DBConection.execute!(conn, query, [ids]) - {rows, query} - end - stop = - fn(conn, reason, query) -> - DBConnection.close!(conn, query, opts) - if reason != :normal do - DBConnection.rollback(conn, reason) - end - end - DBConnection.Stage.consumer_producer(pool, start, handle_events, stop) - """ - @spec producer_consumer(GenServer.server, ((DBConnection.t) -> acc), - ((DBConnection.t, events_in :: [any], acc) -> {events_out :: [any], acc}), - ((DBConnection.t, reason :: any, acc) -> any), Keyword.t) :: - GenServer.on_start when acc: var - def producer_consumer(pool, start, handle_events, stop, opts \\ []) do - start_link(pool, :producer_consumer, start, handle_events, stop, opts) - end - - @doc """ - Start link a `GenStage` consumer that will run a transaction for its duration. - - The first argument is the pool, the second argument is the start function, - the third argument is the handle events function, the fourth argument is the - stop function and the optional fiftth argument are the options. - - The start function is a 1-arity anonymous function with argument - `DBConnection.t`. This is called after the transaction begins but before - `consumer/5` returns. It should return the accumulator or call - `DBConnection.rollback/2` to stop the `GenStage`. - - The handle events function is a 3-arity anonymous function. The first argument - is the `DBConnection.t` for the transaction, the second argument is the - events, and the third argument is the accumulator. This function returns a - 2-tuple, with first element is an empty list (as no outgoing events) and - second element as the accumulator. Also this function can rollback and stop - the `GenStage` using `DBConnection.rollback/2`. - - The stop function is a 3-arity anonymous function. The first argument is the - `DBConnection.t` for the transaction, the second argument is the terminate - reason and the third argument is the accumulator. This function will only be - called if connection is alive and the transaction has not been rolled back. If - this function returns the transaction is committed. This function can rollback - and stop the `GenStage` using `DBConnection.rollback/2`. - For options see "Options" in the module documentation. - The `GenStage` process will behave like a `Flow` stage: - - * It will cancel new and remaining producers when all producers have - notified that they are done or halted and it is a `:consumer` - ### Example - start = - fn(conn) -> - DBConnection.prepare!(conn, query, opts) - end - handle_events = - fn(conn, ids, query) -> - DBConection.execute!(conn, query, [ids]) - {[], query} - end - stop = - fn(conn, reason, query) -> - DBConnection.close!(conn, query, opts) - if reason != :normal do - DBConnection.rollback(conn, reason) - end - end - DBConnection.Stage.consumer(pool, start, handle_events, stop) + query = %Query{statement: "SELECT id FROM table"} + opts = [stream_map: &Map.fetch!(&1, :rows)] + {:ok, stage} = DBConnection.Stage.start_link(pool, query, [], opts) + stage |> GenStage.stream() |> Enum.to_list() """ - @spec consumer(GenServer.server, ((DBConnection.t) -> acc), - ((DBConnection.t, events_in :: [any], acc) -> {[], acc}), - ((DBConnection.t, reason :: any, acc) -> any), Keyword.t) :: - GenServer.on_start when acc: var - def consumer(pool, start, handle_events, stop, opts \\ []) do - start_link(pool, :consumer, start, handle_events, stop, opts) + def start_link(pool, query, params, opts \\ []) do + start_opts = Keyword.take(opts, @start_opts) + args = {pool, query, params, opts} + GenStage.start_link(__MODULE__, args, start_opts) end @doc false - def init({pool, type, start, handle, stop, opts}) do + def init({pool, query, params, opts}) do stage_opts = Keyword.take(opts, @stage_opts) conn = DBConnection.stage_begin(pool, opts) - case DBConnection.stage_transaction(conn, start, opts) do + declare = &declare(&1, query, params, opts) + case DBConnection.stage_transaction(conn, declare, opts) do {:ok, state} -> - stage = %Stage{conn: conn, handle: handle, stop: stop, state: state, - opts: opts, type: type} - {type, stage, stage_opts} + {:producer, %Stage{conn: conn, state: state, opts: opts}, stage_opts} {:error, reason} -> exit(reason) end end - @doc false - def handle_subscribe(:producer, _, {pid, ref} = from, stage) do - case stage do - %Stage{done?: true} -> - GenStage.cancel(from, :normal, [:noconnect]) - {:manual, stage} - %Stage{done?: false, producers: producers} -> - {:automatic, %Stage{stage | producers: Map.put(producers, ref, pid)}} - end - end - def handle_subscribe(:consumer, _, _, stage) do - {:automatic, stage} - end - - @doc false - def handle_cancel(_, _, %Stage{done?: true} = stage) do - {:noreply, stage} - end - def handle_cancel(_, {_, ref}, %Stage{done?: false, producers: producers} = stage) do - case Map.delete(producers, ref) do - new_producers when new_producers == %{} and producers != %{} -> - GenStage.async_info(self(), :stop) - {:noreply, [], %Stage{stage | done?: true, producers: %{}}} - new_producers -> - {:noreply, [], %Stage{stage | producers: new_producers}} - end - end - @doc false def handle_info(:stop, stage) do {:stop, :normal, stage} end + def handle_info({:fetch, conn, pending}, %Stage{conn: conn} = stage) do + handle_demand(pending, stage) + end def handle_info(_, stage) do {:noreply, [], stage} end @doc false def handle_demand(demand, stage) do - %Stage{conn: conn, handle: handle, state: state, opts: opts} = stage - fun = &handle.(&1, demand, state) - case DBConnection.stage_transaction(conn, fun, opts) do - {:ok, {events, state}} -> - {:noreply, events, %Stage{stage | state: state}} - {:error, reason} -> - exit(reason) - :closed -> - raise DBConnection.ConnectionError, "connection is closed" - end - end - - @doc false - def handle_events(events, _, stage) do - %Stage{conn: conn, handle: handle, state: state, opts: opts} = stage - fun = &handle.(&1, events, state) - case DBConnection.stage_transaction(conn, fun, opts) do + %Stage{conn: conn, state: state, opts: opts} = stage + fetch = &fetch(&1, demand, state, opts) + case DBConnection.stage_transaction(conn, fetch, opts) do + {:ok, {:halt, state}} -> + GenStage.async_info(self(), :stop) + {:noreply, [], %Stage{stage | state: state}} {:ok, {events, state}} -> + # stream_map may produce the desired number of events, i.e. at the end + # of the results so we can close the cursor as soon as possible. + pending = demand - length(events) + _ = if pending > 0, do: send(self(), {:fetch, conn, pending}) {:noreply, events, %Stage{stage | state: state}} {:error, reason} -> exit(reason) @@ -342,19 +106,21 @@ defmodule DBConnection.Stage do @doc false def terminate(reason, stage) do - %Stage{conn: conn, stop: stop, state: state, opts: opts} = stage - fun = &stop.(&1, reason, state) - case DBConnection.stage_transaction(conn, fun, opts) do - {:ok, _} -> + %Stage{conn: conn, state: state, opts: opts} = stage + deallocate = &DBConnection.deallocate(&1, state, opts) + case DBConnection.stage_transaction(conn, deallocate, opts) do + {:ok, _} when reason == :normal -> case DBConnection.stage_commit(conn, opts) do :ok -> :ok {:error, :rollback} -> exit(:rollback) end + {:ok, _} -> + DBConnection.stage_rollback(conn, opts) {:error, new_reason} -> DBConnection.stage_rollback(conn, opts) - if new_reason != reason, do: exit(reason) + if new_reason !== reason, do: exit(reason) :closed -> :ok end @@ -362,56 +128,23 @@ defmodule DBConnection.Stage do ## Helpers - defp stream(pool, stream_fun, query, params, opts) do - start = &stream_start(&1, stream_fun, query, params, opts) - producer(pool, start, &stream_next/3, &stream_stop/3, opts) - end - - defp start_link(pool, type, start, handle, stop, opts) do - start_opts = Keyword.take(opts, @start_opts) - args = {pool, type, start, handle, stop, opts} - GenStage.start_link(__MODULE__, args, start_opts) - end - - defp stream_start(conn, stream_fun, query, params, opts) do - stream = stream_fun.(conn, query, params, opts) - {:suspended, _, cont} = Enumerable.reduce(stream, {:suspend, {0, []}}, &stream_reduce/2) - {:cont, cont} - end - - defp stream_reduce(v, {1, acc}) do - {:suspend, {0, [v | acc]}} - end - defp stream_reduce(v, {n, acc}) do - {:cont, {n-1, [v | acc]}} - end - - defp stream_next(_, n, {:cont, cont}) when n > 0 do - case cont.({:cont, {n, []}}) do - {:suspended, {0, acc}, cont} -> - {Enum.reverse(acc), {:cont, cont}} - {state, {_, acc}} when state in [:halted, :done] -> - GenStage.async_info(self(), :stop) - {Enum.reverse(acc), state} + defp declare(conn, query, params, opts) do + case Keyword.get(opts, :prepare, false) do + true -> + DBConnection.prepare_declare(conn, query, params, opts) + false -> + DBConnection.declare(conn, query, params, opts) end end - defp stream_next(_, _, state) when state in [:halted, :done] do - GenStage.async_info(self(), :stop) - {[], state} - end - defp stream_stop(conn, reason, {:cont, cont}) do - _ = cont.({:halt, {0, []}}) - stream_stop(conn, reason) - end - defp stream_stop(conn, reason, state) when state in [:halted, :done] do - stream_stop(conn, reason) - end - - defp stream_stop(_, :normal) do - :ok - end - defp stream_stop(conn, reason) do - DBConnection.rollback(conn, reason) + defp fetch(conn, demand, state, opts) do + try do + DBConnection.fetch(conn, state, [fetch: demand] ++ opts) + catch + kind, reason -> + stack = System.stacktrace() + DBConnection.deallocate(conn, state, opts) + :erlang.raise(kind, reason, stack) + end end end diff --git a/test/test_support.exs b/test/test_support.exs index 2e9ab30f..b95b4940 100644 --- a/test/test_support.exs +++ b/test/test_support.exs @@ -63,28 +63,8 @@ defmodule TestConnection do DBConnection.close!(pool, query, opts2 ++ unquote(opts)) end - def producer(pool, start, handle, stop, opts2 \\ []) do - opts3 = opts2 ++ unquote(opts) - DBConnection.Stage.producer(pool, start, handle, stop, opts3) - end - - def producer_consumer(pool, start, handle, stop, opts2 \\ []) do - opts3 = opts2 ++ unquote(opts) - DBConnection.Stage.producer_consumer(pool, start, handle, stop, opts3) - end - - def consumer(pool, start, handle, stop, opts2 \\ []) do - opts3 = opts2 ++ unquote(opts) - DBConnection.Stage.consumer(pool, start, handle, stop, opts3) - end - def stream_stage(pool, query, params, opts2 \\ []) do - DBConnection.Stage.stream(pool, query, params, opts2 ++ unquote(opts)) - end - - def prepare_stream_stage(pool, query, params, opts2 \\ []) do - opts3 = opts2 ++ unquote(opts) - DBConnection.Stage.prepare_stream(pool, query, params, opts3) + DBConnection.Stage.start_link(pool, query, params, opts2 ++ unquote(opts)) end defoverridable [start_link: 1] From 0a5ac9d1e927598192d28217303339d589c58dc6 Mon Sep 17 00:00:00 2001 From: James Fish Date: Sat, 27 May 2017 17:18:34 +0100 Subject: [PATCH 10/20] Remove Flow from deps/lock file --- mix.exs | 2 -- mix.lock | 1 - 2 files changed, 3 deletions(-) diff --git a/mix.exs b/mix.exs index 3ea9fd58..8f3ee8d0 100644 --- a/mix.exs +++ b/mix.exs @@ -32,8 +32,6 @@ defmodule DBConnection.Mixfile do {:sbroker, "~> 1.0", [optional: true]}, {:gen_stage, "~> 0.11", [optional: true, github: "elixir-lang/gen_stage", branch: "jv-exit-signals"]}, - {:flow, "~> 0.11", [optional: true, github: "elixir-lang/flow", - branch: "jv-exit-signals"]}, {:ex_doc, "~> 0.12", only: :dev}] end diff --git a/mix.lock b/mix.lock index 33473b27..656cb2fa 100644 --- a/mix.lock +++ b/mix.lock @@ -1,7 +1,6 @@ %{"connection": {:hex, :connection, "1.0.4", "a1cae72211f0eef17705aaededacac3eb30e6625b04a6117c1b2db6ace7d5976", [:mix], []}, "earmark": {:hex, :earmark, "0.2.1", "ba6d26ceb16106d069b289df66751734802777a3cbb6787026dd800ffeb850f3", [:mix], []}, "ex_doc": {:hex, :ex_doc, "0.12.0", "b774aabfede4af31c0301aece12371cbd25995a21bb3d71d66f5c2fe074c603f", [:mix], [{:earmark, "~> 0.2", [hex: :earmark, optional: false]}]}, - "flow": {:git, "https://github.com/elixir-lang/flow.git", "cfdc0d240aa990c5229c70e032595f2b063a1445", [branch: "jv-exit-signals"]}, "gen_stage": {:git, "https://github.com/elixir-lang/gen_stage.git", "966d998c699dec58e0d183ddbaa2b783c46cd375", [branch: "jv-exit-signals"]}, "poolboy": {:hex, :poolboy, "1.5.1", "6b46163901cfd0a1b43d692657ed9d7e599853b3b21b95ae5ae0a777cf9b6ca8", [:rebar], []}, "sbroker": {:hex, :sbroker, "1.0.0", "28ff1b5e58887c5098539f236307b36fe1d3edaa2acff9d6a3d17c2dcafebbd0", [:rebar3], []}} From 280393b0e2714ce96cfbda51505c5fd49a7c7414 Mon Sep 17 00:00:00 2001 From: James Fish Date: Sat, 27 May 2017 17:24:44 +0100 Subject: [PATCH 11/20] Correctly handle transient subscriptions --- integration_test/cases/stage_test.exs | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/integration_test/cases/stage_test.exs b/integration_test/cases/stage_test.exs index d22467c0..8cc6b925 100644 --- a/integration_test/cases/stage_test.exs +++ b/integration_test/cases/stage_test.exs @@ -23,7 +23,7 @@ defmodule StageTest do {:ok, pool} = P.start_link(opts) {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) mon = Process.monitor(stage) - assert [stage] |> GenStage.stream([cancel: :transient]) |> Enum.to_list() == [%R{}, %R{}] + assert [{stage, [cancel: :transient]}] |> GenStage.stream() |> Enum.to_list() == [%R{}, %R{}] assert_receive {:DOWN, ^mon, :process, ^stage, :normal} @@ -55,7 +55,7 @@ defmodule StageTest do {:ok, pool} = P.start_link(opts) {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], [prepare: true] ++ opts) mon = Process.monitor(stage) - assert [stage] |> GenStage.stream([cancel: :transient]) |> Enum.to_list() == [%R{}, %R{}] + assert [{stage, [cancel: :transient]}] |> GenStage.stream() |> Enum.to_list() == [%R{}, %R{}] assert_receive {:DOWN, ^mon, :process, ^stage, :normal} @@ -220,7 +220,7 @@ defmodule StageTest do {:ok, pool} = P.start_link(opts) Process.flag(:trap_exit, true) {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) - catch_exit([stage] |> GenStage.stream([cancel: :transient]) |> Enum.to_list()) + catch_exit([{stage, [cancel: :transient]}] |> GenStage.stream() |> Enum.to_list()) assert_receive {:EXIT, ^stage, {^err, _}} assert [ @@ -253,7 +253,7 @@ defmodule StageTest do {:ok, pool} = P.start_link(opts) Process.flag(:trap_exit, true) {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) - catch_exit([stage] |> GenStage.stream([cancel: :transient]) |> Enum.to_list()) + catch_exit([{stage, [cancel: :transient]}] |> GenStage.stream() |> Enum.to_list()) assert_receive {:EXIT, ^stage, {^err, _}} assert_receive :reconnected @@ -285,7 +285,7 @@ defmodule StageTest do {:ok, pool} = P.start_link(opts) Process.flag(:trap_exit, true) {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) - catch_exit([stage] |> GenStage.stream([cancel: :transient]) |> Enum.to_list()) + catch_exit([{stage, [cancel: :transient]}] |> GenStage.stream() |> Enum.to_list()) assert_receive {:EXIT, ^stage, {^err, _}} assert [ @@ -319,7 +319,7 @@ defmodule StageTest do {:ok, pool} = P.start_link(opts) Process.flag(:trap_exit, true) {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) - catch_exit([stage] |> GenStage.stream([cancel: :transient]) |> Enum.to_list()) + catch_exit([{stage, [cancel: :transient]}] |> GenStage.stream() |> Enum.to_list()) assert_receive {:EXIT, ^stage, {^err, _}} assert_receive :reconnected From d44e83903a7599c44ab75cdff7343409d8c7e527 Mon Sep 17 00:00:00 2001 From: James Fish Date: Sun, 28 May 2017 00:47:35 +0100 Subject: [PATCH 12/20] Make resource_begin/transaction/commit/rollback public --- integration_test/cases/resource_test.exs | 367 +++++++++++++++++++++++ integration_test/cases/stage_test.exs | 22 +- lib/db_connection.ex | 281 +++++++++++++---- lib/db_connection/stage.ex | 37 ++- mix.exs | 2 + mix.lock | 1 + test/test_support.exs | 16 + 7 files changed, 638 insertions(+), 88 deletions(-) create mode 100644 integration_test/cases/resource_test.exs diff --git a/integration_test/cases/resource_test.exs b/integration_test/cases/resource_test.exs new file mode 100644 index 00000000..1e64228d --- /dev/null +++ b/integration_test/cases/resource_test.exs @@ -0,0 +1,367 @@ +defmodule ResourceTest do + use ExUnit.Case, async: true + + alias TestPool, as: P + alias TestAgent, as: A + alias TestQuery, as: Q + alias TestResult, as: R + + test "resource transaction commits after stream resource reduced" do + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, %R{}, :newer_state}, + {:ok, :committed, :newest_state} + ] + {:ok, agent} = A.start_link(stack) + + opts = [agent: agent, parent: self()] + {:ok, pool} = P.start_link(opts) + + start = fn() -> P.resource_begin(pool, opts) end + next = fn(conn) -> + {:ok, res} = P.resource_transaction(conn, fn(conn2) -> + P.execute!(conn2, %Q{}, [:param], opts) + end, opts) + {[res], conn} + end + stop = &P.resource_commit/1 + + assert Stream.resource(start, next, stop) |> Enum.take(1) == [%R{}] + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_execute: [%Q{}, [:param], _, :new_state], + handle_commit: [_, :newer_state] + ] = A.record(agent) + end + + test "resource transaction commits per trigger inside Flow pipeline" do + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, %R{}, :newer_state}, + {:ok, :committed, :newest_state} + ] + {:ok, agent} = A.start_link(stack) + + opts = [agent: agent, parent: self()] + {:ok, pool} = P.start_link(opts) + + assert [[:param]] + |> Flow.from_enumerable() + |> Flow.partition(stages: 1) + |> Flow.reduce(fn() -> {[], P.resource_begin(pool, opts)} end, + fn(params, {acc, conn}) -> + {:ok, res} = P.resource_transaction(conn, fn(conn2) -> + P.execute!(conn2, %Q{}, params, opts) + end) + {[res | acc], conn} + end) + |> Flow.map_state(fn({acc, conn}) -> + P.resource_commit(conn, opts) + Enum.reverse(acc) + end) + |> Enum.to_list() == [%R{}] + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_execute: [%Q{}, [:param], _, :new_state], + handle_commit: [_, :newer_state] + ] = A.record(agent) + end + + test "resource_transaction raises inside run" do + stack = [{:ok, :state}] + {:ok, agent} = A.start_link(stack) + + opts = [agent: agent, parent: self()] + {:ok, pool} = P.start_link(opts) + + assert P.run(pool, fn(conn) -> + assert_raise RuntimeError, "not inside transaction", + fn -> + P.resource_transaction(conn, fn(_) -> flunk "should not run" end, opts) + end + :hello + end, opts) == :hello + + assert [ + connect: [_] + ] = A.record(agent) + end + + test "resource_transaction runs inside transaction" do + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, %R{}, :newer_state}, + {:ok, :commited, :newest_state} + ] + {:ok, agent} = A.start_link(stack) + + opts = [agent: agent, parent: self()] + {:ok, pool} = P.start_link(opts) + + assert P.transaction(pool, fn(conn) -> + assert P.resource_transaction(conn, &P.execute!(&1, %Q{}, [:param], opts), + opts) == {:ok, %R{}} + :hello + end) == {:ok, :hello} + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_execute: [%Q{}, [:param], _, :new_state], + handle_commit: [_, :newer_state] + ] = A.record(agent) + end + + test "resource_transaction rolls back and functions error" do + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, :rolledback, :newest_state} + ] + {:ok, agent} = A.start_link(stack) + + opts = [agent: agent, parent: self()] + {:ok, pool} = P.start_link(opts) + + conn = P.resource_begin(pool, opts) + + assert P.resource_rollback(conn, :oops, opts) == {:error, :oops} + assert P.resource_commit(conn, opts) == {:error, :rollback} + assert P.resource_rollback(conn, :oops, opts) == {:error, :oops} + assert P.resource_transaction(conn, fn(_) -> + flunk "should not fun" + end, opts) == {:error, :rollback} + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_rollback: [_, :new_state] + ] = A.record(agent) + end + + test "resource_transaction runs inside resource_transaction" do + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, %R{}, :newer_state}, + {:ok, :commited, :newest_state} + ] + {:ok, agent} = A.start_link(stack) + + opts = [agent: agent, parent: self()] + {:ok, pool} = P.start_link(opts) + + conn = P.resource_begin(pool, opts) + + assert P.resource_transaction(conn, fn(conn2) -> + assert P.resource_transaction(conn2, fn(conn3) -> + P.execute!(conn3, %Q{}, [:param], opts) + end, opts) == {:ok, %R{}} + :hello + end) == {:ok, :hello} + + assert P.resource_commit(conn, opts) == :ok + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_execute: [%Q{}, [:param], _, :new_state], + handle_commit: [_, :newer_state] + ] = A.record(agent) + end + + test "resource_begin logs error" do + err = RuntimeError.exception("oops") + stack = [ + {:ok, :state}, + {:error, err, :new_state} + ] + {:ok, agent} = A.start_link(stack) + + parent = self() + opts = [agent: agent, parent: parent] + {:ok, pool} = P.start_link(opts) + + log = &send(parent, &1) + assert_raise RuntimeError, "oops", + fn() -> P.resource_begin(pool, [log: log]) end + + assert_received %DBConnection.LogEntry{call: :resource_begin} = entry + assert %{query: :begin, params: nil, result: {:error, ^err}} = entry + assert is_integer(entry.pool_time) + assert entry.pool_time >= 0 + assert is_integer(entry.connection_time) + assert entry.connection_time >= 0 + assert is_nil(entry.decode_time) + + assert [ + connect: [_], + handle_begin: [ _, :state]] = A.record(agent) + end + + test "resource_begin logs raises and rolls back" do + err = RuntimeError.exception("oops") + stack = [ + {:ok, :state}, + {:ok, :raise, :new_state}, + {:ok, :rolledback, :newer_state} + ] + {:ok, agent} = A.start_link(stack) + + parent = self() + opts = [agent: agent, parent: parent] + {:ok, pool} = P.start_link(opts) + + log = fn + %DBConnection.LogEntry{result: {:ok, :raise}} -> raise err + entry -> send(parent, entry) + end + + assert_raise RuntimeError, "oops", + fn() -> P.resource_begin(pool, [log: log]) end + + assert_received %DBConnection.LogEntry{call: :resource_begin} = entry + assert %{query: :rollback, params: nil, result: {:ok, :rolledback}} = entry + assert is_nil(entry.pool_time) + assert is_integer(entry.connection_time) + assert entry.connection_time >= 0 + assert is_nil(entry.decode_time) + + assert [ + connect: [_], + handle_begin: [ _, :state], + handle_rollback: [_, :new_state] + ] = A.record(agent) + end + + test "resource_transaction logs on rollback" do + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, :rolledback, :newer_state} + ] + {:ok, agent} = A.start_link(stack) + + parent = self() + opts = [agent: agent, parent: parent] + {:ok, pool} = P.start_link(opts) + + conn = P.resource_begin(pool, opts) + + log = &send(parent, &1) + assert P.resource_transaction(conn, fn(conn2) -> + P.rollback(conn2, :oops) + end, [log: log]) == {:error, :oops} + + assert_received %DBConnection.LogEntry{call: :resource_transaction} = entry + assert %{query: :rollback, params: nil, result: {:ok, :rolledback}} = entry + assert is_nil(entry.pool_time) + assert is_integer(entry.connection_time) + assert entry.connection_time >= 0 + assert is_nil(entry.decode_time) + + assert [ + connect: [_], + handle_begin: [ _, :state], + handle_rollback: [_, :new_state] + ] = A.record(agent) + end + + test "resource_transaction rolls back on failed transaction" do + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, :rolledback, :newer_state} + ] + {:ok, agent} = A.start_link(stack) + + parent = self() + opts = [agent: agent, parent: parent] + {:ok, pool} = P.start_link(opts) + + conn = P.resource_begin(pool, opts) + + assert P.resource_transaction(conn, fn(conn2) -> + assert P.transaction(conn2, &P.rollback(&1, :oops), opts) == {:error, :oops} + end, opts) == {:error, :rollback} + + assert P.resource_transaction(conn, fn(_) -> + flunk "should not run" + end, opts) == {:error, :rollback} + + assert [ + connect: [_], + handle_begin: [ _, :state], + handle_rollback: [_, :new_state] + ] = A.record(agent) + end + + test "resource_commit logs" do + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, :committed, :newer_state} + ] + {:ok, agent} = A.start_link(stack) + + parent = self() + opts = [agent: agent, parent: parent] + {:ok, pool} = P.start_link(opts) + + conn = P.resource_begin(pool, opts) + + log = &send(parent, &1) + assert P.resource_commit(conn, [log: log]) == :ok + + assert_received %DBConnection.LogEntry{call: :resource_commit} = entry + assert %{query: :commit, params: nil, result: {:ok, :committed}} = entry + assert is_nil(entry.pool_time) + assert is_integer(entry.connection_time) + assert entry.connection_time >= 0 + assert is_nil(entry.decode_time) + + assert [ + connect: [_], + handle_begin: [ _, :state], + handle_commit: [_, :new_state] + ] = A.record(agent) + end + + test "resource_rollback logs" do + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, :rolledback, :newer_state} + ] + {:ok, agent} = A.start_link(stack) + + parent = self() + opts = [agent: agent, parent: parent] + {:ok, pool} = P.start_link(opts) + + conn = P.resource_begin(pool, opts) + + log = &send(parent, &1) + assert P.resource_rollback(conn, :oops, [log: log]) == {:error, :oops} + + assert_received %DBConnection.LogEntry{call: :resource_rollback} = entry + assert %{query: :rollback, params: nil, result: {:ok, :rolledback}} = entry + assert is_nil(entry.pool_time) + assert is_integer(entry.connection_time) + assert entry.connection_time >= 0 + assert is_nil(entry.decode_time) + + assert [ + connect: [_], + handle_begin: [ _, :state], + handle_rollback: [_, :new_state] + ] = A.record(agent) + end +end diff --git a/integration_test/cases/stage_test.exs b/integration_test/cases/stage_test.exs index 8cc6b925..45864602 100644 --- a/integration_test/cases/stage_test.exs +++ b/integration_test/cases/stage_test.exs @@ -7,7 +7,7 @@ defmodule StageTest do alias TestCursor, as: C alias TestResult, as: R - test "start_link with GenStage.stream returns result" do + test "start_link produces result" do stack = [ {:ok, :state}, {:ok, :began, :new_state}, @@ -38,7 +38,7 @@ defmodule StageTest do ] = A.record(agent) end - test "start_link with prepare: true returns result" do + test "start_link with prepare: true produces result" do stack = [ {:ok, :state}, {:ok, :began, :new_state}, @@ -53,7 +53,7 @@ defmodule StageTest do opts = [agent: agent, parent: self()] {:ok, pool} = P.start_link(opts) - {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], [prepare: true] ++ opts) + {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], [stage_prepare: true] ++ opts) mon = Process.monitor(stage) assert [{stage, [cancel: :transient]}] |> GenStage.stream() |> Enum.to_list() == [%R{}, %R{}] @@ -71,7 +71,7 @@ defmodule StageTest do ] = A.record(agent) end - test "stream stops normally after it's done" do + test "stage stops normally after it's done" do stack = [ {:ok, :state}, {:ok, :began, :new_state}, @@ -107,7 +107,7 @@ defmodule StageTest do ] = A.record(agent) end - test "stream rolls back on abnormal exit" do + test "stage rolls back on abnormal exit" do stack = [ {:ok, :state}, {:ok, :began, :new_state}, @@ -136,7 +136,7 @@ defmodule StageTest do ] = A.record(agent) end - test "stream declare disconnects" do + test "stage declare disconnects" do err = RuntimeError.exception("oops") stack = [ {:ok, :state}, @@ -167,7 +167,7 @@ defmodule StageTest do ] = A.record(agent) end - test "stream declare bad return raises and stops" do + test "stage declare bad return raises and stops" do stack = [ fn(opts) -> send(opts[:parent], {:hi, self()}) @@ -203,7 +203,7 @@ defmodule StageTest do {:handle_declare, [%Q{}, [:param], _, :new_state]} | _] = A.record(agent) end - test "stream rolls back if first errors" do + test "stage rolls back if first errors" do err = RuntimeError.exception("oops") stack = [ {:ok, :state}, @@ -233,7 +233,7 @@ defmodule StageTest do ] = A.record(agent) end - test "stream first disconnects" do + test "stage first disconnects" do err = RuntimeError.exception("oops") stack = [ {:ok, :state}, @@ -268,7 +268,7 @@ defmodule StageTest do ] = A.record(agent) end - test "stream rolls back if deallocate errors" do + test "stage rolls back if deallocate errors" do err = RuntimeError.exception("oops") stack = [ {:ok, :state}, @@ -298,7 +298,7 @@ defmodule StageTest do ] = A.record(agent) end - test "stream deallocate disconnects" do + test "stage deallocate disconnects" do err = RuntimeError.exception("oops") stack = [ {:ok, :state}, diff --git a/lib/db_connection.ex b/lib/db_connection.ex index 21b8c247..f41479ad 100644 --- a/lib/db_connection.ex +++ b/lib/db_connection.ex @@ -817,12 +817,10 @@ defmodule DBConnection do case get_info(conn) do {transaction, _} when transaction in [:transaction, :failed] -> throw({:rollback, conn_ref, err}) - {transaction, _, _} when transaction in [:transaction, :failed] -> - throw({:rollback, conn_ref, err}) {:idle, _} -> raise "not inside transaction" - {:idle, _, _} -> - raise "not inside transaction" + {:resource, _} -> + raise "not inside resource transaction" :closed -> raise DBConnection.ConnectionError, "connection is closed" end @@ -920,13 +918,46 @@ defmodule DBConnection do resource(conn, start, &fetch/3, &deallocate/3, opts).(acc, fun) end - ## Stage Transaction + @doc """ + Acquire a lock on a connection and return the connection struct for use with + `resource_transaction/3` calls. + + To use the locked connection call `resource_transaction/3` and run requests + and/or nested run/transactions inside it. If the transaction is rolled back + the connection is checked in. Any calls inside a failed transaction (except + `rollback/2`) will raise until the transaction call returns. All running + `transaction/3` or `resource_transaction/3` calls will return + `{:error, :rollback}` if the transaction failed or connection closed and + `rollback/2` is not called for that `transaction/3` or + `resource_transaction/3`. - @doc false - @spec stage_begin(pool :: GenServer.server, opts :: Keyword.t) :: t - def stage_begin(pool, opts) do - {result, log_info} = stage_begin_meter(pool, opts) - transaction_log(log_info) + ### Options + + * `:pool_timeout` - The maximum time to wait for a reply when making a + synchronous call to the pool (default: `5_000`) + * `:queue` - Whether to block waiting in an internal queue for the + connection's state (boolean, default: `true`) + * `:timeout` - The maximum time that the caller is allowed the + to hold the connection's state (default: `15_000`) + * `:log` - A function to log information about begin, commit and rollback + calls made as part of the transaction, either a 1-arity fun, + `{module, function, args}` with `DBConnection.LogEntry.t` prepended to + `args` or `nil`. See `DBConnection.LogEntry` (default: `nil`) + + The pool and connection module may support other options. All options + are passed to `handle_begin/2`, and `handle_rollback/2`. + + ### Example + + conn = DBConnection.resource_begin(pool, opts) + {:ok, res} = DBConnection.resource_transaction(conn, fn(conn) -> + DBConnection.execute!(conn, "SELECT id FROM table", []) + end) + """ + @spec resource_begin(pool :: GenServer.server, opts :: Keyword.t) :: t + def resource_begin(pool, opts \\ []) do + {result, log_info} = resource_begin_meter(pool, opts) + transaction_log(log_info, :resource_begin) case result do {:ok, conn} -> conn @@ -937,28 +968,90 @@ defmodule DBConnection do end end - @doc false - @spec stage_transaction(t, (t -> result), opts :: Keyword.t) :: - {:ok, result} | {:error, reason :: any} | :closed when result: var - def stage_transaction(conn, fun, opts) do + @doc """ + Run a series of requests using a connection checked out with + `resource_begin/3`, inside a transaction. + + The result of the transaction fun is return inside an `:ok` tuple: + `{:ok, result}`. If rollback returns `{:error, reason}` or if connection + not available `{:error, :rollback}`. + + To use the locked connection call the request with the connection + reference passed as the single argument to the `fun`. If the + connection disconnects all future calls using that connection + reference will fail. + + `run/3`, `transaction/3` and `resource_transaction/3` can be nested inside + `resource_transaction/3`. If a resource transaction is rolled back or `fun` + raises the transaction is rolled back and the connection is checked in. If + `resource_transaction/3` is nested in a `resource_transaction/3` or + `transaction/3` it behaves like `transaction/3`. + + ### Options + + * `:log` - A function to log information about begin, commit and rollback + calls made as part of the transaction, either a 1-arity fun, + `{module, function, args}` with `DBConnection.LogEntry.t` prepended to + `args` or `nil`. See `DBConnection.LogEntry` (default: `nil`) + + The pool and connection module may support other options. All options + are passed to `handle_rollback/2`. + + ### Example + + conn = DBConnection.resource_begin(pool, opts) + {:ok, res} = DBConnection.resource_transaction(conn, fn(conn) -> + DBConnection.execute!(conn, "SELECT id FROM table", []) + end) + DBConnection.resource_commit(conn, opts) + + """ + @spec resource_transaction(t, (t -> result), opts :: Keyword.t) :: + {:ok, result} | {:error, reason :: any | :rollback} when result: var + def resource_transaction(conn, fun, opts \\ []) do case get_info(conn) do {:failed, _} -> - stage_rollback(conn, opts) + {:error, :rollback} {:idle, _} -> raise "not inside transaction" {:transaction, _} -> - stage_nested(conn, fun, opts) + transaction_nested(conn, fun) + {:resource, conn_state} -> + resource_nested(conn, conn_state, fun, opts) :closed -> - :closed + {:error, :rollback} end end - @doc false - @spec stage_commit(t, Keyword.t) :: :ok | {:error, :rollback} - def stage_commit(conn, opts) do + @doc """ + Commit transaction and release the lock on a connection. + + Returns `:ok` on sucess, otherwise `{:error, :rollback}` if the + transaction failed and was rolled back or the connection is not available. + + Can only be called for a connection checked out with `resource_begin/2` when + outside of `resource_transaction/3`. + + ### Options + + * `:log` - A function to log information about begin, commit and rollback + calls made as part of the transaction, either a 1-arity fun, + `{module, function, args}` with `DBConnection.LogEntry.t` prepended to + `args` or `nil`. See `DBConnection.LogEntry` (default: `nil`) + + The pool and connection module may support other options. All options + are passed to `handle_commit/2`. + + ### Example + + conn = DBConnection.resource_begin(pool, opts) + DBConnection.resource_commit(conn, opts) + """ + @spec resource_commit(t, Keyword.t) :: :ok | {:error, :rollback} + def resource_commit(conn, opts \\ []) do log = Keyword.get(opts, :log) - {result, log_info} = stage_conclude(conn, &commit/4, log, opts, :ok) - transaction_log(log_info) + {result, log_info} = resource_conclude(conn, &commit/4, log, opts, :ok) + transaction_log(log_info, :resource_commit) case result do {:raise, err} -> raise err @@ -969,13 +1062,34 @@ defmodule DBConnection do end end - @doc false - @spec stage_rollback(t, Keyword.t) :: {:error, :rollback} - def stage_rollback(conn, opts) do - log = Keyword.get(opts, :log) - result = {:error, :rollback} - {result, log_info} = stage_conclude(conn, &rollback/4, log, opts, result) - transaction_log(log_info) + @doc """ + Rollback transaction and release the lock on a connection. + + Returns `{:error, reason}` if rolls back or connection not available. + + Can only be called for a connection checked out with `resource_begin/2` when + outside of `resource_transaction/3`. + + ### Options + + * `:log` - A function to log information about begin, commit and rollback + calls made as part of the transaction, either a 1-arity fun, + `{module, function, args}` with `DBConnection.LogEntry.t` prepended to + `args` or `nil`. See `DBConnection.LogEntry` (default: `nil`) + + The pool and connection module may support other options. All options + are passed to `handle_rollback/2`. + + ### Example + + conn = DBConnection.resource_begin(pool, opts) + DBConnection.resource_rollback(conn, reason, opts) + """ + @spec resource_rollback(t, reason, Keyword.t) :: + {:error, reason} when reason: var + def resource_rollback(conn, reason, opts \\ []) do + {result, log_info} = run_resource_rollback(conn, reason, opts) + transaction_log(log_info, :resource_rollback) case result do {:raise, err} -> raise err @@ -1225,11 +1339,13 @@ defmodule DBConnection do log(call, query, params, log, [decode: time()] ++ times, result) end - defp transaction_log(nil), do: :ok - defp transaction_log({log, times, callback, result}) do + defp transaction_log(log, fun \\ :transaction) + + defp transaction_log(nil, _fun), do: :ok + defp transaction_log({log, times, callback, result}, fun) do call = transaction_call(callback) result = transaction_result(result) - _ = log(:transaction, call, nil, log, times, result) + _ = log(fun, call, nil, log, times, result) :ok end @@ -1362,7 +1478,7 @@ defmodule DBConnection do defp commit(conn, log, opts, result) do case get_info(conn) do - {:transaction, conn_state} -> + {trans, conn_state} when trans in [:transaction, :resource] -> conclude_meter(conn, conn_state, log, :handle_commit, opts, result) {:failed, conn_state} -> result = {:error, :rollback} @@ -1374,7 +1490,7 @@ defmodule DBConnection do defp rollback(conn, log, opts, result) do case get_info(conn) do - {trans, conn_state} when trans in [:transaction, :failed] -> + {trans, conn_state} when trans in [:transaction, :failed, :resource] -> conclude_meter(conn, conn_state, log, :handle_rollback, opts, result) :closed -> {result, nil} @@ -1556,7 +1672,7 @@ defmodule DBConnection do {:halt, state} end - def run_fetch(conn, fun, call, query, cursor, opts) do + defp run_fetch(conn, fun, call, query, cursor, opts) do fetch = &handle(&1, fun, [query, cursor], opts) case run_meter(conn, fetch, opts) do {{:ok, result}, meter} -> @@ -1609,24 +1725,24 @@ defmodule DBConnection do Stream.resource(start, next, stop) end - defp stage_begin_meter(pool, opts) do + defp resource_begin_meter(pool, opts) do case Keyword.get(opts, :log) do nil -> - stage_begin_meter(pool, nil, [], opts) + resource_begin_meter(pool, nil, [], opts) log -> times = [checkout: time()] - stage_begin_meter(pool, log, times, opts) + resource_begin_meter(pool, log, times, opts) end end - defp stage_begin_meter(pool, log, times, opts) do + defp resource_begin_meter(pool, log, times, opts) do {conn, conn_state} = checkout(pool, opts) put_info(conn, :idle, conn_state) - stage_begin_meter(conn, conn_state, log, times, opts) + resource_begin_meter(conn, conn_state, log, times, opts) end - defp stage_begin_meter(conn, conn_state, nil, [], opts) do - case handle(conn, conn_state, :handle_begin, opts, :transaction) do + defp resource_begin_meter(conn, conn_state, nil, [], opts) do + case handle(conn, conn_state, :handle_begin, opts, :resource) do {:ok, _} -> {{:ok, conn}, nil} error -> @@ -1634,56 +1750,105 @@ defmodule DBConnection do {error, nil} end end - defp stage_begin_meter(conn, conn_state, log, times, opts) do + defp resource_begin_meter(conn, conn_state, log, times, opts) do start = time() - result = handle(conn, conn_state, :handle_begin, opts, :transaction) + result = handle(conn, conn_state, :handle_begin, opts, :resource) times = [stop: time(), start: start] ++ times log_info = {log, times, :handle_begin, result} case result do {:ok, _} -> - stage_begin_log(conn, log_info, opts) + resource_begin_log(conn, log_info, opts) error -> run_end(conn, opts) {error, log_info} end end - defp stage_begin_log(conn, {log, _, _, _} = log_info, opts) do + defp resource_begin_log(conn, {log, _, _, _} = log_info, opts) do try do - transaction_log(log_info) + transaction_log(log_info, :resource_begin) catch kind, reason -> result = {kind, reason, System.stacktrace()} - stage_conclude(conn, &rollback/4, log, opts, result) + resource_conclude(conn, &rollback/4, log, opts, result) else _ -> {{:ok, conn}, nil} end end - defp stage_nested(conn, fun, opts) do + defp resource_nested(conn, state, fun, opts) do %DBConnection{conn_ref: conn_ref} = conn try do + put_info(conn, :transaction, state) fun.(conn) catch :throw, {:rollback, ^conn_ref, reason} -> - stage_rollback(conn, opts) - {:error, reason} + resource_failed(conn, reason, opts) kind, reason -> stack = System.stacktrace() - stage_rollback(conn, opts) + resource_failed(conn, :raise, opts) :erlang.raise(kind, reason, stack) else result -> + resource_ok(conn, result, opts) + end + end + + defp resource_ok(conn, result, opts) do + case get_info(conn) do + {:transaction, conn_state} -> + put_info(conn, :resource, conn_state) {:ok, result} + {:failed, conn_state} -> + put_info(conn, :resource, conn_state) + resource_nested_rollback(conn, :rollback, opts) + _ -> + {:error, :rollback} end end - defp stage_conclude(conn, fun, log, opts, result) do - try do - fun.(conn, log, opts, result) - after - run_end(conn, opts) + defp resource_failed(conn, reason, opts) do + case get_info(conn) do + {trans, conn_state} when trans in [:transaction, :failed] -> + put_info(conn, :resource, conn_state) + resource_nested_rollback(conn, reason, opts) + :closed -> + {:error, reason} + end + end + + defp resource_nested_rollback(conn, reason, opts) do + {result, log_info} = run_resource_rollback(conn, reason, opts) + transaction_log(log_info, :resource_transaction) + case result do + {:raise, err} -> + raise err + {kind, reason, stack} -> + :erlang.raise(kind, reason, stack) + other -> + other + end + end + + defp run_resource_rollback(conn, reason, opts) do + log = Keyword.get(opts, :log) + result = {:error, reason} + resource_conclude(conn, &rollback/4, log, opts, result, result) + end + + defp resource_conclude(conn, fun, log, opts, result, closed \\ {:error, :rollback}) do + case get_info(conn) do + {:resource, _} -> + try do + fun.(conn, log, opts, result) + after + run_end(conn, opts) + end + {trans, _} when trans in [:transaction, :failed] -> + raise "inside transaction" + :closed -> + {closed, nil} end end @@ -1696,6 +1861,8 @@ defmodule DBConnection do case get_info(conn) do {:failed, _} -> raise DBConnection.ConnectionError, "transaction rolling back" + {:resource, _} -> + raise "not inside resource transaction" {_, _} = info -> info :closed -> diff --git a/lib/db_connection/stage.ex b/lib/db_connection/stage.ex index 6e3faf0d..4c24e8b9 100644 --- a/lib/db_connection/stage.ex +++ b/lib/db_connection/stage.ex @@ -8,7 +8,7 @@ defmodule DBConnection.Stage do * `:stream_map` - A function to flat map the results of the query, either a 2-arity fun, `{module, function, args}` with `DBConnection.t` and the result prepended to `args` or `nil` (default: `nil`) - * `:prepare` - Whether the consumer should prepare the query before + * `:stage_prepare` - Whether the consumer should prepare the query before streaming it (default: `false`) * `:pool_timeout` - The maximum time to wait for a reply when making a synchronous call to the pool (default: `5_000`) @@ -62,9 +62,9 @@ defmodule DBConnection.Stage do @doc false def init({pool, query, params, opts}) do stage_opts = Keyword.take(opts, @stage_opts) - conn = DBConnection.stage_begin(pool, opts) + conn = DBConnection.resource_begin(pool, opts) declare = &declare(&1, query, params, opts) - case DBConnection.stage_transaction(conn, declare, opts) do + case DBConnection.resource_transaction(conn, declare, opts) do {:ok, state} -> {:producer, %Stage{conn: conn, state: state, opts: opts}, stage_opts} {:error, reason} -> @@ -87,7 +87,7 @@ defmodule DBConnection.Stage do def handle_demand(demand, stage) do %Stage{conn: conn, state: state, opts: opts} = stage fetch = &fetch(&1, demand, state, opts) - case DBConnection.stage_transaction(conn, fetch, opts) do + case DBConnection.resource_transaction(conn, fetch, opts) do {:ok, {:halt, state}} -> GenStage.async_info(self(), :stop) {:noreply, [], %Stage{stage | state: state}} @@ -107,21 +107,13 @@ defmodule DBConnection.Stage do @doc false def terminate(reason, stage) do %Stage{conn: conn, state: state, opts: opts} = stage - deallocate = &DBConnection.deallocate(&1, state, opts) - case DBConnection.stage_transaction(conn, deallocate, opts) do - {:ok, _} when reason == :normal -> - case DBConnection.stage_commit(conn, opts) do - :ok -> - :ok - {:error, :rollback} -> - exit(:rollback) - end - {:ok, _} -> - DBConnection.stage_rollback(conn, opts) - {:error, new_reason} -> - DBConnection.stage_rollback(conn, opts) - if new_reason !== reason, do: exit(reason) - :closed -> + deallocate = &deallocate(&1, reason, state, opts) + case DBConnection.resource_transaction(conn, deallocate, opts) do + {:ok, :normal} -> + DBConnection.resource_commit(conn, opts) + {:ok, reason} -> + DBConnection.resource_rollback(conn, reason, opts) + {:error, :rollback} -> :ok end end @@ -129,7 +121,7 @@ defmodule DBConnection.Stage do ## Helpers defp declare(conn, query, params, opts) do - case Keyword.get(opts, :prepare, false) do + case Keyword.get(opts, :stage_prepare, false) do true -> DBConnection.prepare_declare(conn, query, params, opts) false -> @@ -147,4 +139,9 @@ defmodule DBConnection.Stage do :erlang.raise(kind, reason, stack) end end + + defp deallocate(conn, reason, state, opts) do + :ok = DBConnection.deallocate(conn, state, opts) + reason + end end diff --git a/mix.exs b/mix.exs index 8f3ee8d0..3ea9fd58 100644 --- a/mix.exs +++ b/mix.exs @@ -32,6 +32,8 @@ defmodule DBConnection.Mixfile do {:sbroker, "~> 1.0", [optional: true]}, {:gen_stage, "~> 0.11", [optional: true, github: "elixir-lang/gen_stage", branch: "jv-exit-signals"]}, + {:flow, "~> 0.11", [optional: true, github: "elixir-lang/flow", + branch: "jv-exit-signals"]}, {:ex_doc, "~> 0.12", only: :dev}] end diff --git a/mix.lock b/mix.lock index 656cb2fa..33473b27 100644 --- a/mix.lock +++ b/mix.lock @@ -1,6 +1,7 @@ %{"connection": {:hex, :connection, "1.0.4", "a1cae72211f0eef17705aaededacac3eb30e6625b04a6117c1b2db6ace7d5976", [:mix], []}, "earmark": {:hex, :earmark, "0.2.1", "ba6d26ceb16106d069b289df66751734802777a3cbb6787026dd800ffeb850f3", [:mix], []}, "ex_doc": {:hex, :ex_doc, "0.12.0", "b774aabfede4af31c0301aece12371cbd25995a21bb3d71d66f5c2fe074c603f", [:mix], [{:earmark, "~> 0.2", [hex: :earmark, optional: false]}]}, + "flow": {:git, "https://github.com/elixir-lang/flow.git", "cfdc0d240aa990c5229c70e032595f2b063a1445", [branch: "jv-exit-signals"]}, "gen_stage": {:git, "https://github.com/elixir-lang/gen_stage.git", "966d998c699dec58e0d183ddbaa2b783c46cd375", [branch: "jv-exit-signals"]}, "poolboy": {:hex, :poolboy, "1.5.1", "6b46163901cfd0a1b43d692657ed9d7e599853b3b21b95ae5ae0a777cf9b6ca8", [:rebar], []}, "sbroker": {:hex, :sbroker, "1.0.0", "28ff1b5e58887c5098539f236307b36fe1d3edaa2acff9d6a3d17c2dcafebbd0", [:rebar3], []}} diff --git a/test/test_support.exs b/test/test_support.exs index b95b4940..01867dac 100644 --- a/test/test_support.exs +++ b/test/test_support.exs @@ -67,6 +67,22 @@ defmodule TestConnection do DBConnection.Stage.start_link(pool, query, params, opts2 ++ unquote(opts)) end + def resource_begin(pool, opts2 \\ []) do + DBConnection.resource_begin(pool, opts2 ++ unquote(opts)) + end + + def resource_transaction(conn, fun, opts2 \\ []) do + DBConnection.resource_transaction(conn, fun, opts2 ++ unquote(opts)) + end + + def resource_commit(conn, opts2 \\ []) do + DBConnection.resource_commit(conn, opts2 ++ unquote(opts)) + end + + def resource_rollback(conn, reason, opts2 \\ []) do + DBConnection.resource_rollback(conn, reason, opts2 ++ unquote(opts)) + end + defoverridable [start_link: 1] end end From d59b6d67b9222fafaa73efdbd083a0300ba5cc75 Mon Sep 17 00:00:00 2001 From: James Fish Date: Sun, 28 May 2017 02:53:38 +0100 Subject: [PATCH 13/20] Add support for stage without transaction --- integration_test/cases/resource_test.exs | 66 +++- integration_test/cases/stage_test.exs | 182 +++++----- .../cases/transaction_stage_test.exs | 337 ++++++++++++++++++ integration_test/tests.exs | 2 + lib/db_connection.ex | 134 ++++--- lib/db_connection/stage.ex | 74 ++-- test/test_support.exs | 4 + 7 files changed, 635 insertions(+), 164 deletions(-) create mode 100644 integration_test/cases/transaction_stage_test.exs diff --git a/integration_test/cases/resource_test.exs b/integration_test/cases/resource_test.exs index 1e64228d..abb2eb7a 100644 --- a/integration_test/cases/resource_test.exs +++ b/integration_test/cases/resource_test.exs @@ -93,6 +93,70 @@ defmodule ResourceTest do ] = A.record(agent) end + test "resource_begin raises on checkin" do + stack = [ + fn(opts) -> + Process.link(opts[:parent]) + {:ok, :state} + end, + {:ok, :began, :new_state}, + {:ok, :state2} + ] + {:ok, agent} = A.start_link(stack) + + opts = [agent: agent, parent: self()] + Process.flag(:trap_exit, true) + {:ok, pool} = P.start_link(opts) + + conn = P.resource_begin(pool, opts) + + assert_raise RuntimeError, "inside transaction", + fn() -> P.checkin(conn, opts) end + + assert P.resource_commit(conn, opts) == {:error, :rollback} + + assert_receive {:EXIT, _, {%DBConnection.ConnectionError{}, [_|_]}} + + assert [ + {:connect, [_]}, + {:handle_begin, [_, :state]} | _] = A.record(agent) + end + + test "resource_transaction raises on checkin" do + stack = [ + fn(opts) -> + Process.link(opts[:parent]) + {:ok, :state} + end, + {:ok, :began, :new_state}, + {:ok, :state2} + ] + {:ok, agent} = A.start_link(stack) + + opts = [agent: agent, parent: self()] + Process.flag(:trap_exit, true) + {:ok, pool} = P.start_link(opts) + + conn = P.resource_begin(pool, opts) + + assert_raise RuntimeError, "inside transaction", + fn() -> P.checkin(conn, opts) end + + assert_receive {:EXIT, _, {%DBConnection.ConnectionError{}, [_|_]}} + + assert P.resource_transaction(conn, fn(conn2) -> + assert_raise RuntimeError, "inside transaction", + fn() -> P.checkin(conn2, opts) end + :hello + end) == {:error, :rollback} + + assert P.resource_commit(conn, opts) == {:error, :rollback} + + assert [ + {:connect, [_]}, + {:handle_begin, [_, :state]} | _] = A.record(agent) + end + test "resource_transaction runs inside transaction" do stack = [ {:ok, :state}, @@ -119,7 +183,7 @@ defmodule ResourceTest do ] = A.record(agent) end - test "resource_transaction rolls back and functions error" do + test "resource_transaction rolls back and returns error" do stack = [ {:ok, :state}, {:ok, :began, :new_state}, diff --git a/integration_test/cases/stage_test.exs b/integration_test/cases/stage_test.exs index 45864602..5fb0557c 100644 --- a/integration_test/cases/stage_test.exs +++ b/integration_test/cases/stage_test.exs @@ -10,16 +10,14 @@ defmodule StageTest do test "start_link produces result" do stack = [ {:ok, :state}, - {:ok, :began, :new_state}, - {:ok, %C{}, :newer_state}, - {:ok, %R{}, :newest_state}, - {:deallocate, %R{}, :state2}, - {:ok, :deallocated, :new_state2}, - {:ok, :commited, :newer_state2} + {:ok, %C{}, :new_state}, + {:ok, %R{}, :newer_state}, + {:deallocate, %R{}, :newest_state}, + {:ok, :deallocated, :state2}, ] {:ok, agent} = A.start_link(stack) - opts = [agent: agent, parent: self()] + opts = [agent: agent, parent: self(), stage_transaction: false] {:ok, pool} = P.start_link(opts) {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) mon = Process.monitor(stage) @@ -29,29 +27,25 @@ defmodule StageTest do assert [ connect: [_], - handle_begin: [_, :state], - handle_declare: [%Q{}, [:param], _, :new_state], - handle_first: [%Q{}, %C{}, _, :newer_state], - handle_next: [%Q{}, %C{}, _, :newest_state], - handle_deallocate: [%Q{}, %C{}, _, :state2], - handle_commit: [_, :new_state2] + handle_declare: [%Q{}, [:param], _, :state], + handle_first: [%Q{}, %C{}, _, :new_state], + handle_next: [%Q{}, %C{}, _, :newer_state], + handle_deallocate: [%Q{}, %C{}, _, :newest_state] ] = A.record(agent) end test "start_link with prepare: true produces result" do stack = [ {:ok, :state}, - {:ok, :began, :new_state}, - {:ok, %Q{}, :newer_state}, - {:ok, %C{}, :newest_state}, - {:ok, %R{}, :state2}, - {:deallocate, %R{}, :new_state2}, - {:ok, :deallocated, :newer_state2}, - {:ok, :commited, :newest_state2} + {:ok, %Q{}, :new_state}, + {:ok, %C{}, :newer_state}, + {:ok, %R{}, :newest_state}, + {:deallocate, %R{}, :state2}, + {:ok, :deallocated, :new_state2}, ] {:ok, agent} = A.start_link(stack) - opts = [agent: agent, parent: self()] + opts = [agent: agent, parent: self(), stage_transaction: false] {:ok, pool} = P.start_link(opts) {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], [stage_prepare: true] ++ opts) mon = Process.monitor(stage) @@ -61,29 +55,25 @@ defmodule StageTest do assert [ connect: [_], - handle_begin: [_, :state], - handle_prepare: [%Q{}, _, :new_state], - handle_declare: [%Q{}, [:param], _, :newer_state], - handle_first: [%Q{}, %C{}, _, :newest_state], - handle_next: [%Q{}, %C{}, _, :state2], - handle_deallocate: [%Q{}, %C{}, _, :new_state2], - handle_commit: [_, :newer_state2] + handle_prepare: [%Q{}, _, :state], + handle_declare: [%Q{}, [:param], _, :new_state], + handle_first: [%Q{}, %C{}, _, :newer_state], + handle_next: [%Q{}, %C{}, _, :newest_state], + handle_deallocate: [%Q{}, %C{}, _, :state2] ] = A.record(agent) end test "stage stops normally after it's done" do stack = [ {:ok, :state}, - {:ok, :began, :new_state}, - {:ok, %C{}, :newer_state}, - {:deallocate, %R{}, :state2}, - {:ok, :deallocated, :new_state2}, - {:ok, :commited, :newer_state2} + {:ok, %C{}, :new_state}, + {:deallocate, %R{}, :newer_state}, + {:ok, :deallocated, :newest_state}, ] {:ok, agent} = A.start_link(stack) parent = self() - opts = [agent: agent, parent: parent] + opts = [agent: agent, parent: parent, stage_transaction: false] {:ok, pool} = P.start_link(opts) {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) @@ -99,26 +89,23 @@ defmodule StageTest do assert [ connect: [_], - handle_begin: [_, :state], - handle_declare: [%Q{}, [:param], _, :new_state], - handle_first: [%Q{}, %C{}, _, :newer_state], - handle_deallocate: [%Q{}, %C{}, _, :state2], - handle_commit: [_, :new_state2] + handle_declare: [%Q{}, [:param], _, :state], + handle_first: [%Q{}, %C{}, _, :new_state], + handle_deallocate: [%Q{}, %C{}, _, :newer_state], ] = A.record(agent) end - test "stage rolls back on abnormal exit" do + test "stage checks in on abnormal exit" do stack = [ {:ok, :state}, - {:ok, :began, :new_state}, - {:ok, %C{}, :newer_state}, - {:ok, :deallocated, :newest_state}, - {:ok, :rolledback, :state2} + {:ok, %C{}, :new_state}, + {:ok, :deallocated, :newer_state}, + {:ok, %R{}, :new_state2} ] {:ok, agent} = A.start_link(stack) parent = self() - opts = [agent: agent, parent: parent] + opts = [agent: agent, parent: parent, stage_transaction: false] Process.flag(:trap_exit, true) {:ok, pool} = P.start_link(opts) {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) @@ -127,12 +114,13 @@ defmodule StageTest do GenStage.stop(stage, :oops) + assert P.execute!(pool, %Q{}, [:param], opts) == %R{} + assert [ connect: [_], - handle_begin: [_, :state], - handle_declare: [%Q{}, [:param], _, :new_state], - handle_deallocate: [%Q{}, %C{}, _, :newer_state], - handle_rollback: [_, :newest_state] + handle_declare: [%Q{}, [:param], _, :state], + handle_deallocate: [%Q{}, %C{}, _, :new_state], + handle_execute: [%Q{}, [:param], _, :newer_state] ] = A.record(agent) end @@ -140,8 +128,7 @@ defmodule StageTest do err = RuntimeError.exception("oops") stack = [ {:ok, :state}, - {:ok, :began, :new_state}, - {:disconnect, err, :newer_state}, + {:disconnect, err, :new_state}, :ok, fn(opts) -> send(opts[:parent], :reconnected) @@ -151,7 +138,7 @@ defmodule StageTest do {:ok, agent} = A.start_link(stack) parent = self() - opts = [agent: agent, parent: parent] + opts = [agent: agent, parent: parent, stage_transaction: false] Process.flag(:trap_exit, true) {:ok, pool} = P.start_link(opts) assert {:error, {^err, _}} = P.stream_stage(pool, %Q{}, [:param], opts) @@ -160,9 +147,8 @@ defmodule StageTest do assert [ connect: [_], - handle_begin: [_, :state], - handle_declare: [%Q{}, [:param], _, :new_state], - disconnect: [^err, :newer_state], + handle_declare: [%Q{}, [:param], _, :state], + disconnect: [^err, :new_state], connect: [_] ] = A.record(agent) end @@ -174,14 +160,13 @@ defmodule StageTest do Process.link(opts[:parent]) {:ok, :state} end, - {:ok, :began, :new_state}, :oops, {:ok, :state2} ] {:ok, agent} = A.start_link(stack) parent = self() - opts = [agent: agent, parent: parent] + opts = [agent: agent, parent: parent, stage_transaction: false] {:ok, pool} = P.start_link(opts) assert_receive {:hi, conn} @@ -199,37 +184,36 @@ defmodule StageTest do assert [ {:connect, _}, - {:handle_begin, [_, :state]}, - {:handle_declare, [%Q{}, [:param], _, :new_state]} | _] = A.record(agent) + {:handle_declare, [%Q{}, [:param], _, :state]} | _] = A.record(agent) end - test "stage rolls back if first errors" do + test "stage checks in if first errors" do err = RuntimeError.exception("oops") stack = [ {:ok, :state}, - {:ok, :began, :new_state}, - {:ok, %C{}, :newer_state}, - {:error, err, :newest_state}, - {:ok, :deallocated, :state2}, - {:ok, :rolledback, :new_state2} + {:ok, %C{}, :new_state}, + {:error, err, :newer_state}, + {:ok, :deallocated, :newest_state}, + {:ok, %R{}, :state2} ] {:ok, agent} = A.start_link(stack) parent = self() - opts = [agent: agent, parent: parent] + opts = [agent: agent, parent: parent, stage_transaction: false] {:ok, pool} = P.start_link(opts) Process.flag(:trap_exit, true) {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) catch_exit([{stage, [cancel: :transient]}] |> GenStage.stream() |> Enum.to_list()) assert_receive {:EXIT, ^stage, {^err, _}} + assert P.execute!(pool, %Q{}, [:param], opts) == %R{} + assert [ connect: [_], - handle_begin: [_, :state], - handle_declare: [%Q{}, [:param], _, :new_state], - handle_first: [%Q{}, %C{}, _, :newer_state], - handle_deallocate: [%Q{}, %C{}, _, :newest_state], - handle_rollback: [_, :state2] + handle_declare: [%Q{}, [:param], _, :state], + handle_first: [%Q{}, %C{}, _, :new_state], + handle_deallocate: [%Q{}, %C{}, _, :newer_state], + handle_execute: [%Q{}, [:param], _, :newest_state] ] = A.record(agent) end @@ -237,9 +221,8 @@ defmodule StageTest do err = RuntimeError.exception("oops") stack = [ {:ok, :state}, - {:ok, :began, :new_state}, - {:ok, %C{}, :newer_state}, - {:disconnect, err, :newest_state}, + {:ok, %C{}, :new_state}, + {:disconnect, err, :newer_state}, :ok, fn(opts) -> send(opts[:parent], :reconnected) @@ -249,7 +232,7 @@ defmodule StageTest do {:ok, agent} = A.start_link(stack) parent = self() - opts = [agent: agent, parent: parent] + opts = [agent: agent, parent: parent, stage_transaction: false] {:ok, pool} = P.start_link(opts) Process.flag(:trap_exit, true) {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) @@ -260,41 +243,40 @@ defmodule StageTest do assert [ connect: [_], - handle_begin: [_, :state], - handle_declare: [%Q{}, [:param], _, :new_state], - handle_first: [%Q{}, %C{}, _, :newer_state], - disconnect: [^err, :newest_state], + handle_declare: [%Q{}, [:param], _, :state], + handle_first: [%Q{}, %C{}, _, :new_state], + disconnect: [^err, :newer_state], connect: [_] ] = A.record(agent) end - test "stage rolls back if deallocate errors" do + test "stage checks in if deallocate errors" do err = RuntimeError.exception("oops") stack = [ {:ok, :state}, - {:ok, :began, :new_state}, - {:ok, %C{}, :newer_state}, - {:deallocate, %R{}, :newest_state}, - {:error, err, :state2}, - {:ok, :rolledback, :new_state2} + {:ok, %C{}, :new_state}, + {:deallocate, %R{}, :newer_state}, + {:error, err, :newest_state}, + {:ok, %R{}, :state2} ] {:ok, agent} = A.start_link(stack) parent = self() - opts = [agent: agent, parent: parent] + opts = [agent: agent, parent: parent, stage_transaction: false] {:ok, pool} = P.start_link(opts) Process.flag(:trap_exit, true) {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) catch_exit([{stage, [cancel: :transient]}] |> GenStage.stream() |> Enum.to_list()) assert_receive {:EXIT, ^stage, {^err, _}} + assert P.execute!(pool, %Q{}, [:param], opts) == %R{} + assert [ connect: [_], - handle_begin: [_, :state], - handle_declare: [%Q{}, [:param], _, :new_state], - handle_first: [%Q{}, %C{}, _, :newer_state], - handle_deallocate: [%Q{}, %C{}, _, :newest_state], - handle_rollback: [_, :state2] + handle_declare: [%Q{}, [:param], _, :state], + handle_first: [%Q{}, %C{}, _, :new_state], + handle_deallocate: [%Q{}, %C{}, _, :newer_state], + handle_execute: [%Q{}, [:param], _, :newest_state] ] = A.record(agent) end @@ -302,10 +284,9 @@ defmodule StageTest do err = RuntimeError.exception("oops") stack = [ {:ok, :state}, - {:ok, :began, :new_state}, - {:ok, %C{}, :newer_state}, - {:deallocate, %R{}, :newest_state}, - {:disconnect, err, :state2}, + {:ok, %C{}, :new_state}, + {:deallocate, %R{}, :newer_state}, + {:disconnect, err, :newest_state}, :ok, fn(opts) -> send(opts[:parent], :reconnected) @@ -315,7 +296,7 @@ defmodule StageTest do {:ok, agent} = A.start_link(stack) parent = self() - opts = [agent: agent, parent: parent] + opts = [agent: agent, parent: parent, stage_transaction: false] {:ok, pool} = P.start_link(opts) Process.flag(:trap_exit, true) {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) @@ -326,11 +307,10 @@ defmodule StageTest do assert [ connect: [_], - handle_begin: [_, :state], - handle_declare: [%Q{}, [:param], _, :new_state], - handle_first: [%Q{}, %C{}, _, :newer_state], - handle_deallocate: [%Q{}, %C{}, _, :newest_state], - disconnect: [^err, :state2], + handle_declare: [%Q{}, [:param], _, :state], + handle_first: [%Q{}, %C{}, _, :new_state], + handle_deallocate: [%Q{}, %C{}, _, :newer_state], + disconnect: [^err, :newest_state], connect: [_] ] = A.record(agent) end diff --git a/integration_test/cases/transaction_stage_test.exs b/integration_test/cases/transaction_stage_test.exs new file mode 100644 index 00000000..ae770dc1 --- /dev/null +++ b/integration_test/cases/transaction_stage_test.exs @@ -0,0 +1,337 @@ +defmodule TransactionStageTest do + use ExUnit.Case, async: true + + alias TestPool, as: P + alias TestAgent, as: A + alias TestQuery, as: Q + alias TestCursor, as: C + alias TestResult, as: R + + test "start_link produces result" do + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, %C{}, :newer_state}, + {:ok, %R{}, :newest_state}, + {:deallocate, %R{}, :state2}, + {:ok, :deallocated, :new_state2}, + {:ok, :commited, :newer_state2} + ] + {:ok, agent} = A.start_link(stack) + + opts = [agent: agent, parent: self()] + {:ok, pool} = P.start_link(opts) + {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) + mon = Process.monitor(stage) + assert [{stage, [cancel: :transient]}] |> GenStage.stream() |> Enum.to_list() == [%R{}, %R{}] + + assert_receive {:DOWN, ^mon, :process, ^stage, :normal} + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_declare: [%Q{}, [:param], _, :new_state], + handle_first: [%Q{}, %C{}, _, :newer_state], + handle_next: [%Q{}, %C{}, _, :newest_state], + handle_deallocate: [%Q{}, %C{}, _, :state2], + handle_commit: [_, :new_state2] + ] = A.record(agent) + end + + test "start_link with prepare: true produces result" do + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, %Q{}, :newer_state}, + {:ok, %C{}, :newest_state}, + {:ok, %R{}, :state2}, + {:deallocate, %R{}, :new_state2}, + {:ok, :deallocated, :newer_state2}, + {:ok, :commited, :newest_state2} + ] + {:ok, agent} = A.start_link(stack) + + opts = [agent: agent, parent: self()] + {:ok, pool} = P.start_link(opts) + {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], [stage_prepare: true] ++ opts) + mon = Process.monitor(stage) + assert [{stage, [cancel: :transient]}] |> GenStage.stream() |> Enum.to_list() == [%R{}, %R{}] + + assert_receive {:DOWN, ^mon, :process, ^stage, :normal} + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_prepare: [%Q{}, _, :new_state], + handle_declare: [%Q{}, [:param], _, :newer_state], + handle_first: [%Q{}, %C{}, _, :newest_state], + handle_next: [%Q{}, %C{}, _, :state2], + handle_deallocate: [%Q{}, %C{}, _, :new_state2], + handle_commit: [_, :newer_state2] + ] = A.record(agent) + end + + test "stage stops normally after it's done" do + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, %C{}, :newer_state}, + {:deallocate, %R{}, :state2}, + {:ok, :deallocated, :new_state2}, + {:ok, :commited, :newer_state2} + ] + {:ok, agent} = A.start_link(stack) + + parent = self() + opts = [agent: agent, parent: parent] + {:ok, pool} = P.start_link(opts) + {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) + + ref = Process.monitor(stage) + send(stage, {:"$gen_producer", {parent, ref}, {:subscribe, nil, [cancel: :transient]}}) + sub = {stage, ref} + GenStage.ask(sub, 1000) + + assert_receive {:"$gen_consumer", ^sub, [%R{}]} + + assert_receive {:DOWN, ^ref, :process, ^stage, :normal} + refute_received {:"$gen_producer", ^sub, _} + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_declare: [%Q{}, [:param], _, :new_state], + handle_first: [%Q{}, %C{}, _, :newer_state], + handle_deallocate: [%Q{}, %C{}, _, :state2], + handle_commit: [_, :new_state2] + ] = A.record(agent) + end + + test "stage rolls back on abnormal exit" do + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, %C{}, :newer_state}, + {:ok, :deallocated, :newest_state}, + {:ok, :rolledback, :state2} + ] + {:ok, agent} = A.start_link(stack) + + parent = self() + opts = [agent: agent, parent: parent] + Process.flag(:trap_exit, true) + {:ok, pool} = P.start_link(opts) + {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) + + send(stage, {:"$gen_producer", {parent, make_ref()}, {:subscribe, nil, []}}) + + GenStage.stop(stage, :oops) + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_declare: [%Q{}, [:param], _, :new_state], + handle_deallocate: [%Q{}, %C{}, _, :newer_state], + handle_rollback: [_, :newest_state] + ] = A.record(agent) + end + + test "stage declare disconnects" do + err = RuntimeError.exception("oops") + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:disconnect, err, :newer_state}, + :ok, + fn(opts) -> + send(opts[:parent], :reconnected) + {:ok, :state2} + end + ] + {:ok, agent} = A.start_link(stack) + + parent = self() + opts = [agent: agent, parent: parent] + Process.flag(:trap_exit, true) + {:ok, pool} = P.start_link(opts) + assert {:error, {^err, _}} = P.stream_stage(pool, %Q{}, [:param], opts) + + assert_receive :reconnected + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_declare: [%Q{}, [:param], _, :new_state], + disconnect: [^err, :newer_state], + connect: [_] + ] = A.record(agent) + end + + test "stage declare bad return raises and stops" do + stack = [ + fn(opts) -> + send(opts[:parent], {:hi, self()}) + Process.link(opts[:parent]) + {:ok, :state} + end, + {:ok, :began, :new_state}, + :oops, + {:ok, :state2} + ] + {:ok, agent} = A.start_link(stack) + + parent = self() + opts = [agent: agent, parent: parent] + {:ok, pool} = P.start_link(opts) + assert_receive {:hi, conn} + + Process.flag(:trap_exit, true) + {:error, {%DBConnection.ConnectionError{}, _} = reason} = P.stream_stage(pool, %Q{}, [:param], opts) + + assert_receive {:EXIT, stage, ^reason} + + prefix = "client #{inspect stage} stopped: " <> + "** (DBConnection.ConnectionError) bad return value: :oops" + len = byte_size(prefix) + assert_receive {:EXIT, ^conn, + {%DBConnection.ConnectionError{message: <<^prefix::binary-size(len), _::binary>>}, + [_|_]}} + + assert [ + {:connect, _}, + {:handle_begin, [_, :state]}, + {:handle_declare, [%Q{}, [:param], _, :new_state]} | _] = A.record(agent) + end + + test "stage rolls back if first errors" do + err = RuntimeError.exception("oops") + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, %C{}, :newer_state}, + {:error, err, :newest_state}, + {:ok, :deallocated, :state2}, + {:ok, :rolledback, :new_state2} + ] + {:ok, agent} = A.start_link(stack) + + parent = self() + opts = [agent: agent, parent: parent] + {:ok, pool} = P.start_link(opts) + Process.flag(:trap_exit, true) + {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) + catch_exit([{stage, [cancel: :transient]}] |> GenStage.stream() |> Enum.to_list()) + assert_receive {:EXIT, ^stage, {^err, _}} + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_declare: [%Q{}, [:param], _, :new_state], + handle_first: [%Q{}, %C{}, _, :newer_state], + handle_deallocate: [%Q{}, %C{}, _, :newest_state], + handle_rollback: [_, :state2] + ] = A.record(agent) + end + + test "stage first disconnects" do + err = RuntimeError.exception("oops") + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, %C{}, :newer_state}, + {:disconnect, err, :newest_state}, + :ok, + fn(opts) -> + send(opts[:parent], :reconnected) + {:ok, :state2} + end + ] + {:ok, agent} = A.start_link(stack) + + parent = self() + opts = [agent: agent, parent: parent] + {:ok, pool} = P.start_link(opts) + Process.flag(:trap_exit, true) + {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) + catch_exit([{stage, [cancel: :transient]}] |> GenStage.stream() |> Enum.to_list()) + assert_receive {:EXIT, ^stage, {^err, _}} + + assert_receive :reconnected + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_declare: [%Q{}, [:param], _, :new_state], + handle_first: [%Q{}, %C{}, _, :newer_state], + disconnect: [^err, :newest_state], + connect: [_] + ] = A.record(agent) + end + + test "stage rolls back if deallocate errors" do + err = RuntimeError.exception("oops") + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, %C{}, :newer_state}, + {:deallocate, %R{}, :newest_state}, + {:error, err, :state2}, + {:ok, :rolledback, :new_state2} + ] + {:ok, agent} = A.start_link(stack) + + parent = self() + opts = [agent: agent, parent: parent] + {:ok, pool} = P.start_link(opts) + Process.flag(:trap_exit, true) + {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) + catch_exit([{stage, [cancel: :transient]}] |> GenStage.stream() |> Enum.to_list()) + assert_receive {:EXIT, ^stage, {^err, _}} + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_declare: [%Q{}, [:param], _, :new_state], + handle_first: [%Q{}, %C{}, _, :newer_state], + handle_deallocate: [%Q{}, %C{}, _, :newest_state], + handle_rollback: [_, :state2] + ] = A.record(agent) + end + + test "stage deallocate disconnects" do + err = RuntimeError.exception("oops") + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, %C{}, :newer_state}, + {:deallocate, %R{}, :newest_state}, + {:disconnect, err, :state2}, + :ok, + fn(opts) -> + send(opts[:parent], :reconnected) + {:ok, :new_state2} + end + ] + {:ok, agent} = A.start_link(stack) + + parent = self() + opts = [agent: agent, parent: parent] + {:ok, pool} = P.start_link(opts) + Process.flag(:trap_exit, true) + {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) + catch_exit([{stage, [cancel: :transient]}] |> GenStage.stream() |> Enum.to_list()) + assert_receive {:EXIT, ^stage, {^err, _}} + + assert_receive :reconnected + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_declare: [%Q{}, [:param], _, :new_state], + handle_first: [%Q{}, %C{}, _, :newer_state], + handle_deallocate: [%Q{}, %C{}, _, :newest_state], + disconnect: [^err, :state2], + connect: [_] + ] = A.record(agent) + end +end diff --git a/integration_test/tests.exs b/integration_test/tests.exs index da99142e..2f9546e0 100644 --- a/integration_test/tests.exs +++ b/integration_test/tests.exs @@ -9,7 +9,9 @@ Code.require_file "cases/prepare_execute_test.exs", __DIR__ Code.require_file "cases/prepare_stream_test.exs", __DIR__ Code.require_file "cases/prepare_test.exs", __DIR__ Code.require_file "cases/queue_test.exs", __DIR__ +Code.require_file "cases/resource_test.exs", __DIR__ Code.require_file "cases/stage_test.exs", __DIR__ Code.require_file "cases/stream_test.exs", __DIR__ Code.require_file "cases/transaction_execute_test.exs", __DIR__ +Code.require_file "cases/transaction_stage_test.exs", __DIR__ Code.require_file "cases/transaction_test.exs", __DIR__ diff --git a/lib/db_connection.ex b/lib/db_connection.ex index f41479ad..15b8f37f 100644 --- a/lib/db_connection.ex +++ b/lib/db_connection.ex @@ -739,9 +739,12 @@ defmodule DBConnection do fun.(conn) end def run(pool, fun, opts) do - {conn, conn_state} = checkout(pool, opts) - put_info(conn, :idle, conn_state) - run_begin(conn, fun, opts) + conn = checkout(pool, opts) + try do + fun.(conn) + after + checkin(conn, opts) + end end @doc """ @@ -918,6 +921,79 @@ defmodule DBConnection do resource(conn, start, &fetch/3, &deallocate/3, opts).(acc, fun) end + @doc """ + Acquire a lock on a connection and return the connection struct for use with + `run/3` and/or `transaction/3` calls. + + `run/3` and `transaction/3` can be nested multiple times but a + `transaction/3` call inside another `transaction/3` will be treated + the same as `run/3`. + + ### Options + + * `:pool_timeout` - The maximum time to wait for a reply when making a + synchronous call to the pool (default: `5_000`) + * `:queue` - Whether to block waiting in an internal queue for the + connection's state (boolean, default: `true`) + * `:timeout` - The maximum time that the caller is allowed the + to hold the connection's state (default: `15_000`) + + The pool may support other options. + + ### Example + + conn = DBConnection.checkout(pool) + try do + DBConnection.execute!(conn, "SELECT id FROM table", []) + after + DBConnection.checkin(conn) + end + """ + @spec checkout(pool :: GenServer.server, opts :: Keyword.t) :: t + def checkout(pool, opts \\ []) do + case run_checkout(pool, opts) do + {:ok, conn, _} -> + conn + {:error, err} -> + raise err + end + end + + @doc """ + Release the lock on a connection. + + Returns `:ok`. + + The pool may support options. + + ### Example + + conn = DBConnection.checkout(pool) + try do + DBConnection.execute!(conn, "SELECT id FROM table", []) + after + DBConnection.checkin(conn) + end + """ + @spec checkin(conn :: t, opts :: Keyword.t) :: :ok + def checkin(%DBConnection{} = conn, opts \\ []) do + case delete_info(conn) do + {:idle, conn_state} -> + run_checkin(conn, conn_state, opts) + {status, conn_state} when status in [:transaction, :failed, :resource] -> + try do + raise "inside transaction" + catch + :error, reason -> + stack = System.stacktrace() + delete_stop(conn, conn_state, :error, reason, stack, opts) + :erlang.raise(:error, reason, stack) + end + :closed -> + :ok + end + end + @doc """ Acquire a lock on a connection and return the connection struct for use with `resource_transaction/3` calls. @@ -1102,19 +1178,20 @@ defmodule DBConnection do ## Helpers - defp checkout(pool, opts) do + defp run_checkout(pool, opts) do pool_mod = Keyword.get(opts, :pool, DBConnection.Connection) case apply(pool_mod, :checkout, [pool, opts]) do {:ok, pool_ref, conn_mod, conn_state} -> conn = %DBConnection{pool_mod: pool_mod, pool_ref: pool_ref, conn_mod: conn_mod, conn_ref: make_ref()} - {conn, conn_state} - {:error, err} -> - raise err + put_info(conn, :idle, conn_state) + {:ok, conn, conn_state} + {:error, _} = error -> + error end end - defp checkin(conn, conn_state, opts) do + defp run_checkin(conn, conn_state, opts) do %DBConnection{pool_mod: pool_mod, pool_ref: pool_ref} = conn _ = apply(pool_mod, :checkin, [pool_ref, conn_state, opts]) :ok @@ -1383,32 +1460,6 @@ defmodule DBConnection do end defp log_result(other), do: other - defp run_begin(conn, fun, opts) do - try do - fun.(conn) - after - run_end(conn, opts) - end - end - - defp run_end(conn, opts) do - case delete_info(conn) do - {:idle, conn_state} -> - checkin(conn, conn_state, opts) - {status, conn_state} when status in [:transaction, :failed] -> - try do - raise "connection run ended in transaction" - catch - :error, reason -> - stack = System.stacktrace() - delete_stop(conn, conn_state, :error, reason, stack, opts) - :erlang.raise(:error, reason, stack) - end - :closed -> - :ok - end - end - defp transaction_meter(%DBConnection{} = conn, fun, opts) do case fetch_info(conn) do {:transaction, _} -> @@ -1736,9 +1787,12 @@ defmodule DBConnection do end defp resource_begin_meter(pool, log, times, opts) do - {conn, conn_state} = checkout(pool, opts) - put_info(conn, :idle, conn_state) - resource_begin_meter(conn, conn_state, log, times, opts) + case run_checkout(pool, opts) do + {:ok, conn, conn_state} -> + resource_begin_meter(conn, conn_state, log, times, opts) + {:error, err} -> + {:raise, err} + end end defp resource_begin_meter(conn, conn_state, nil, [], opts) do @@ -1746,7 +1800,7 @@ defmodule DBConnection do {:ok, _} -> {{:ok, conn}, nil} error -> - run_end(conn, opts) + checkin(conn, opts) {error, nil} end end @@ -1759,7 +1813,7 @@ defmodule DBConnection do {:ok, _} -> resource_begin_log(conn, log_info, opts) error -> - run_end(conn, opts) + checkin(conn, opts) {error, log_info} end end @@ -1843,7 +1897,7 @@ defmodule DBConnection do try do fun.(conn, log, opts, result) after - run_end(conn, opts) + checkin(conn, opts) end {trans, _} when trans in [:transaction, :failed] -> raise "inside transaction" diff --git a/lib/db_connection/stage.ex b/lib/db_connection/stage.ex index 4c24e8b9..c56976b7 100644 --- a/lib/db_connection/stage.ex +++ b/lib/db_connection/stage.ex @@ -1,15 +1,17 @@ defmodule DBConnection.Stage do @moduledoc """ - A `GenStage` producer that encapsulates a transaction and streams the - result of a query. + A `GenStage` producer that streams the result of a query, optionally + encapsulated in a transaction. ### Options * `:stream_map` - A function to flat map the results of the query, either a 2-arity fun, `{module, function, args}` with `DBConnection.t` and the result prepended to `args` or `nil` (default: `nil`) - * `:stage_prepare` - Whether the consumer should prepare the query before + * `:stage_prepare` - Whether the producer should prepare the query before streaming it (default: `false`) + * `:stage_transaction` - Whether the producer should encapsulate the query + in a transaction (default: `true`) * `:pool_timeout` - The maximum time to wait for a reply when making a synchronous call to the pool (default: `5_000`) * `:queue` - Whether to block waiting in an internal queue for the @@ -31,8 +33,8 @@ defmodule DBConnection.Stage do use GenStage - @enforce_keys [:conn, :state, :opts] - defstruct [:conn, :state, :opts] + @enforce_keys [:conn, :state, :transaction?, :opts] + defstruct [:conn, :state, :transaction?, :opts] @start_opts [:name, :spawn_opt, :debug] @stage_opts [:demand, :buffer_size, :buffer_keep, :dispatcher] @@ -62,14 +64,9 @@ defmodule DBConnection.Stage do @doc false def init({pool, query, params, opts}) do stage_opts = Keyword.take(opts, @stage_opts) - conn = DBConnection.resource_begin(pool, opts) - declare = &declare(&1, query, params, opts) - case DBConnection.resource_transaction(conn, declare, opts) do - {:ok, state} -> - {:producer, %Stage{conn: conn, state: state, opts: opts}, stage_opts} - {:error, reason} -> - exit(reason) - end + stage = init(pool, opts) + state = run(&declare(&1, query, params, opts), opts, stage) + {:producer, %Stage{stage | state: state}, stage_opts} end @doc false @@ -86,26 +83,21 @@ defmodule DBConnection.Stage do @doc false def handle_demand(demand, stage) do %Stage{conn: conn, state: state, opts: opts} = stage - fetch = &fetch(&1, demand, state, opts) - case DBConnection.resource_transaction(conn, fetch, opts) do - {:ok, {:halt, state}} -> + case run(&fetch(&1, demand, state, opts), opts, stage) do + {:halt, state} -> GenStage.async_info(self(), :stop) {:noreply, [], %Stage{stage | state: state}} - {:ok, {events, state}} -> + {events, state} -> # stream_map may produce the desired number of events, i.e. at the end # of the results so we can close the cursor as soon as possible. pending = demand - length(events) _ = if pending > 0, do: send(self(), {:fetch, conn, pending}) {:noreply, events, %Stage{stage | state: state}} - {:error, reason} -> - exit(reason) - :closed -> - raise DBConnection.ConnectionError, "connection is closed" end end @doc false - def terminate(reason, stage) do + def terminate(reason, %Stage{transaction?: true} = stage) do %Stage{conn: conn, state: state, opts: opts} = stage deallocate = &deallocate(&1, reason, state, opts) case DBConnection.resource_transaction(conn, deallocate, opts) do @@ -117,9 +109,47 @@ defmodule DBConnection.Stage do :ok end end + def terminate(reason, %Stage{transaction?: false} = stage) do + %Stage{conn: conn, state: state, opts: opts} = stage + try do + deallocate(conn, reason, state, opts) + after + DBConnection.checkin(conn, opts) + end + end ## Helpers + defp init(pool, opts) do + case Keyword.get(opts, :stage_transaction, true) do + true -> + conn = DBConnection.resource_begin(pool, opts) + %Stage{conn: conn, transaction?: true, state: :declare, opts: opts} + false -> + conn = DBConnection.checkout(pool, opts) + %Stage{conn: conn, transaction?: false, state: :declare, opts: opts} + end + end + + defp run(fun, opts, %Stage{conn: conn, transaction?: true}) do + case DBConnection.resource_transaction(conn, fun, opts) do + {:ok, result} -> + result + {:error, reason} -> + exit(reason) + end + end + defp run(fun, opts, %Stage{conn: conn, transaction?: false}) do + try do + fun.(conn) + catch + kind, reason -> + stack = System.stacktrace() + DBConnection.checkin(conn, opts) + :erlang.raise(kind, reason, stack) + end + end + defp declare(conn, query, params, opts) do case Keyword.get(opts, :stage_prepare, false) do true -> diff --git a/test/test_support.exs b/test/test_support.exs index 01867dac..7fd7c796 100644 --- a/test/test_support.exs +++ b/test/test_support.exs @@ -83,6 +83,10 @@ defmodule TestConnection do DBConnection.resource_rollback(conn, reason, opts2 ++ unquote(opts)) end + def checkin(conn, opts2 \\ []) do + DBConnection.checkin(conn, opts2 ++ unquote(opts)) + end + defoverridable [start_link: 1] end end From 3514e04efae5bd5dc28fb6d18f87d9f367e9d20c Mon Sep 17 00:00:00 2001 From: James Fish Date: Sun, 28 May 2017 13:25:06 +0100 Subject: [PATCH 14/20] Use same transaction function for continuation or normal --- ...esource_test.exs => continuation_test.exs} | 122 ++++---- integration_test/cases/transaction_test.exs | 6 +- integration_test/tests.exs | 2 +- lib/db_connection.ex | 288 +++++++----------- lib/db_connection/stage.ex | 10 +- test/test_support.exs | 16 +- 6 files changed, 183 insertions(+), 261 deletions(-) rename integration_test/cases/{resource_test.exs => continuation_test.exs} (72%) diff --git a/integration_test/cases/resource_test.exs b/integration_test/cases/continuation_test.exs similarity index 72% rename from integration_test/cases/resource_test.exs rename to integration_test/cases/continuation_test.exs index abb2eb7a..05336d26 100644 --- a/integration_test/cases/resource_test.exs +++ b/integration_test/cases/continuation_test.exs @@ -1,4 +1,4 @@ -defmodule ResourceTest do +defmodule ContinuationTest do use ExUnit.Case, async: true alias TestPool, as: P @@ -6,7 +6,7 @@ defmodule ResourceTest do alias TestQuery, as: Q alias TestResult, as: R - test "resource transaction commits after stream resource reduced" do + test "transaction commits after stream resource reduced" do stack = [ {:ok, :state}, {:ok, :began, :new_state}, @@ -18,14 +18,14 @@ defmodule ResourceTest do opts = [agent: agent, parent: self()] {:ok, pool} = P.start_link(opts) - start = fn() -> P.resource_begin(pool, opts) end + start = fn() -> P.checkout_begin(pool, opts) end next = fn(conn) -> - {:ok, res} = P.resource_transaction(conn, fn(conn2) -> + {:ok, res} = P.transaction(conn, fn(conn2) -> P.execute!(conn2, %Q{}, [:param], opts) end, opts) {[res], conn} end - stop = &P.resource_commit/1 + stop = &P.commit_checkin/1 assert Stream.resource(start, next, stop) |> Enum.take(1) == [%R{}] @@ -37,7 +37,7 @@ defmodule ResourceTest do ] = A.record(agent) end - test "resource transaction commits per trigger inside Flow pipeline" do + test "transaction commits per trigger inside Flow pipeline" do stack = [ {:ok, :state}, {:ok, :began, :new_state}, @@ -52,15 +52,15 @@ defmodule ResourceTest do assert [[:param]] |> Flow.from_enumerable() |> Flow.partition(stages: 1) - |> Flow.reduce(fn() -> {[], P.resource_begin(pool, opts)} end, + |> Flow.reduce(fn() -> {[], P.checkout_begin(pool, opts)} end, fn(params, {acc, conn}) -> - {:ok, res} = P.resource_transaction(conn, fn(conn2) -> + {:ok, res} = P.transaction(conn, fn(conn2) -> P.execute!(conn2, %Q{}, params, opts) end) {[res | acc], conn} end) |> Flow.map_state(fn({acc, conn}) -> - P.resource_commit(conn, opts) + P.commit_checkin(conn, opts) Enum.reverse(acc) end) |> Enum.to_list() == [%R{}] @@ -73,27 +73,7 @@ defmodule ResourceTest do ] = A.record(agent) end - test "resource_transaction raises inside run" do - stack = [{:ok, :state}] - {:ok, agent} = A.start_link(stack) - - opts = [agent: agent, parent: self()] - {:ok, pool} = P.start_link(opts) - - assert P.run(pool, fn(conn) -> - assert_raise RuntimeError, "not inside transaction", - fn -> - P.resource_transaction(conn, fn(_) -> flunk "should not run" end, opts) - end - :hello - end, opts) == :hello - - assert [ - connect: [_] - ] = A.record(agent) - end - - test "resource_begin raises on checkin" do + test "checkout_begin raises on checkin" do stack = [ fn(opts) -> Process.link(opts[:parent]) @@ -108,12 +88,12 @@ defmodule ResourceTest do Process.flag(:trap_exit, true) {:ok, pool} = P.start_link(opts) - conn = P.resource_begin(pool, opts) + conn = P.checkout_begin(pool, opts) assert_raise RuntimeError, "inside transaction", fn() -> P.checkin(conn, opts) end - assert P.resource_commit(conn, opts) == {:error, :rollback} + assert P.commit_checkin(conn, opts) == {:error, :rollback} assert_receive {:EXIT, _, {%DBConnection.ConnectionError{}, [_|_]}} @@ -122,7 +102,7 @@ defmodule ResourceTest do {:handle_begin, [_, :state]} | _] = A.record(agent) end - test "resource_transaction raises on checkin" do + test "transaction raises on checkin" do stack = [ fn(opts) -> Process.link(opts[:parent]) @@ -137,27 +117,27 @@ defmodule ResourceTest do Process.flag(:trap_exit, true) {:ok, pool} = P.start_link(opts) - conn = P.resource_begin(pool, opts) + conn = P.checkout_begin(pool, opts) assert_raise RuntimeError, "inside transaction", fn() -> P.checkin(conn, opts) end assert_receive {:EXIT, _, {%DBConnection.ConnectionError{}, [_|_]}} - assert P.resource_transaction(conn, fn(conn2) -> + assert P.transaction(conn, fn(conn2) -> assert_raise RuntimeError, "inside transaction", fn() -> P.checkin(conn2, opts) end :hello end) == {:error, :rollback} - assert P.resource_commit(conn, opts) == {:error, :rollback} + assert P.commit_checkin(conn, opts) == {:error, :rollback} assert [ {:connect, [_]}, {:handle_begin, [_, :state]} | _] = A.record(agent) end - test "resource_transaction runs inside transaction" do + test "transaction runs inside transaction" do stack = [ {:ok, :state}, {:ok, :began, :new_state}, @@ -170,7 +150,7 @@ defmodule ResourceTest do {:ok, pool} = P.start_link(opts) assert P.transaction(pool, fn(conn) -> - assert P.resource_transaction(conn, &P.execute!(&1, %Q{}, [:param], opts), + assert P.transaction(conn, &P.execute!(&1, %Q{}, [:param], opts), opts) == {:ok, %R{}} :hello end) == {:ok, :hello} @@ -183,7 +163,7 @@ defmodule ResourceTest do ] = A.record(agent) end - test "resource_transaction rolls back and returns error" do + test "transaction rolls back and returns error" do stack = [ {:ok, :state}, {:ok, :began, :new_state}, @@ -194,12 +174,12 @@ defmodule ResourceTest do opts = [agent: agent, parent: self()] {:ok, pool} = P.start_link(opts) - conn = P.resource_begin(pool, opts) + conn = P.checkout_begin(pool, opts) - assert P.resource_rollback(conn, :oops, opts) == {:error, :oops} - assert P.resource_commit(conn, opts) == {:error, :rollback} - assert P.resource_rollback(conn, :oops, opts) == {:error, :oops} - assert P.resource_transaction(conn, fn(_) -> + assert P.rollback_checkin(conn, :oops, opts) == {:error, :oops} + assert P.commit_checkin(conn, opts) == {:error, :rollback} + assert P.rollback_checkin(conn, :oops, opts) == {:error, :oops} + assert P.transaction(conn, fn(_) -> flunk "should not fun" end, opts) == {:error, :rollback} @@ -210,7 +190,7 @@ defmodule ResourceTest do ] = A.record(agent) end - test "resource_transaction runs inside resource_transaction" do + test "transaction runs inside resource_transaction" do stack = [ {:ok, :state}, {:ok, :began, :new_state}, @@ -222,16 +202,16 @@ defmodule ResourceTest do opts = [agent: agent, parent: self()] {:ok, pool} = P.start_link(opts) - conn = P.resource_begin(pool, opts) + conn = P.checkout_begin(pool, opts) - assert P.resource_transaction(conn, fn(conn2) -> - assert P.resource_transaction(conn2, fn(conn3) -> + assert P.transaction(conn, fn(conn2) -> + assert P.transaction(conn2, fn(conn3) -> P.execute!(conn3, %Q{}, [:param], opts) end, opts) == {:ok, %R{}} :hello end) == {:ok, :hello} - assert P.resource_commit(conn, opts) == :ok + assert P.commit_checkin(conn, opts) == :ok assert [ connect: [_], @@ -241,7 +221,7 @@ defmodule ResourceTest do ] = A.record(agent) end - test "resource_begin logs error" do + test "checkout_begin logs error" do err = RuntimeError.exception("oops") stack = [ {:ok, :state}, @@ -255,9 +235,9 @@ defmodule ResourceTest do log = &send(parent, &1) assert_raise RuntimeError, "oops", - fn() -> P.resource_begin(pool, [log: log]) end + fn() -> P.checkout_begin(pool, [log: log]) end - assert_received %DBConnection.LogEntry{call: :resource_begin} = entry + assert_received %DBConnection.LogEntry{call: :checkout_begin} = entry assert %{query: :begin, params: nil, result: {:error, ^err}} = entry assert is_integer(entry.pool_time) assert entry.pool_time >= 0 @@ -270,7 +250,7 @@ defmodule ResourceTest do handle_begin: [ _, :state]] = A.record(agent) end - test "resource_begin logs raises and rolls back" do + test "checkout_begin logs raises and rolls back" do err = RuntimeError.exception("oops") stack = [ {:ok, :state}, @@ -289,9 +269,9 @@ defmodule ResourceTest do end assert_raise RuntimeError, "oops", - fn() -> P.resource_begin(pool, [log: log]) end + fn() -> P.checkout_begin(pool, [log: log]) end - assert_received %DBConnection.LogEntry{call: :resource_begin} = entry + assert_received %DBConnection.LogEntry{call: :checkout_begin} = entry assert %{query: :rollback, params: nil, result: {:ok, :rolledback}} = entry assert is_nil(entry.pool_time) assert is_integer(entry.connection_time) @@ -305,7 +285,7 @@ defmodule ResourceTest do ] = A.record(agent) end - test "resource_transaction logs on rollback" do + test "transaction logs on rollback" do stack = [ {:ok, :state}, {:ok, :began, :new_state}, @@ -317,14 +297,14 @@ defmodule ResourceTest do opts = [agent: agent, parent: parent] {:ok, pool} = P.start_link(opts) - conn = P.resource_begin(pool, opts) + conn = P.checkout_begin(pool, opts) log = &send(parent, &1) - assert P.resource_transaction(conn, fn(conn2) -> + assert P.transaction(conn, fn(conn2) -> P.rollback(conn2, :oops) end, [log: log]) == {:error, :oops} - assert_received %DBConnection.LogEntry{call: :resource_transaction} = entry + assert_received %DBConnection.LogEntry{call: :transaction} = entry assert %{query: :rollback, params: nil, result: {:ok, :rolledback}} = entry assert is_nil(entry.pool_time) assert is_integer(entry.connection_time) @@ -338,7 +318,7 @@ defmodule ResourceTest do ] = A.record(agent) end - test "resource_transaction rolls back on failed transaction" do + test "transaction rolls back on failed transaction" do stack = [ {:ok, :state}, {:ok, :began, :new_state}, @@ -350,13 +330,13 @@ defmodule ResourceTest do opts = [agent: agent, parent: parent] {:ok, pool} = P.start_link(opts) - conn = P.resource_begin(pool, opts) + conn = P.checkout_begin(pool, opts) - assert P.resource_transaction(conn, fn(conn2) -> + assert P.transaction(conn, fn(conn2) -> assert P.transaction(conn2, &P.rollback(&1, :oops), opts) == {:error, :oops} end, opts) == {:error, :rollback} - assert P.resource_transaction(conn, fn(_) -> + assert P.transaction(conn, fn(_) -> flunk "should not run" end, opts) == {:error, :rollback} @@ -367,7 +347,7 @@ defmodule ResourceTest do ] = A.record(agent) end - test "resource_commit logs" do + test "commit_checkin logs" do stack = [ {:ok, :state}, {:ok, :began, :new_state}, @@ -379,12 +359,12 @@ defmodule ResourceTest do opts = [agent: agent, parent: parent] {:ok, pool} = P.start_link(opts) - conn = P.resource_begin(pool, opts) + conn = P.checkout_begin(pool, opts) log = &send(parent, &1) - assert P.resource_commit(conn, [log: log]) == :ok + assert P.commit_checkin(conn, [log: log]) == :ok - assert_received %DBConnection.LogEntry{call: :resource_commit} = entry + assert_received %DBConnection.LogEntry{call: :commit_checkin} = entry assert %{query: :commit, params: nil, result: {:ok, :committed}} = entry assert is_nil(entry.pool_time) assert is_integer(entry.connection_time) @@ -398,7 +378,7 @@ defmodule ResourceTest do ] = A.record(agent) end - test "resource_rollback logs" do + test "rollback_checkin logs" do stack = [ {:ok, :state}, {:ok, :began, :new_state}, @@ -410,12 +390,12 @@ defmodule ResourceTest do opts = [agent: agent, parent: parent] {:ok, pool} = P.start_link(opts) - conn = P.resource_begin(pool, opts) + conn = P.checkout_begin(pool, opts) log = &send(parent, &1) - assert P.resource_rollback(conn, :oops, [log: log]) == {:error, :oops} + assert P.rollback_checkin(conn, :oops, [log: log]) == {:error, :oops} - assert_received %DBConnection.LogEntry{call: :resource_rollback} = entry + assert_received %DBConnection.LogEntry{call: :rollback_checkin} = entry assert %{query: :rollback, params: nil, result: {:ok, :rolledback}} = entry assert is_nil(entry.pool_time) assert is_integer(entry.connection_time) diff --git a/integration_test/cases/transaction_test.exs b/integration_test/cases/transaction_test.exs index 1576f2d5..69ff378e 100644 --- a/integration_test/cases/transaction_test.exs +++ b/integration_test/cases/transaction_test.exs @@ -135,8 +135,7 @@ defmodule TransactionTest do P.rollback(conn2, :oops) end) == {:error, :oops} - assert_raise DBConnection.ConnectionError, "transaction rolling back", - fn() -> P.transaction(conn, fn(_) -> nil end) end + assert P.transaction(conn, fn(_) -> nil end) == {:error, :rollback} end) == {:error, :rollback} assert P.transaction(pool, fn(_) -> :result end) == {:ok, :result} @@ -197,8 +196,7 @@ defmodule TransactionTest do assert_raise RuntimeError, "oops", fn() -> P.transaction(conn, fn(_) -> raise "oops" end) end - assert_raise DBConnection.ConnectionError, "transaction rolling back", - fn() -> P.transaction(conn, fn(_) -> nil end) end + assert P.transaction(conn, fn(_) -> nil end) == {:error, :rollback} end) == {:error, :rollback} assert P.transaction(pool, fn(_) -> :result end) == {:ok, :result} diff --git a/integration_test/tests.exs b/integration_test/tests.exs index 2f9546e0..4f42d19e 100644 --- a/integration_test/tests.exs +++ b/integration_test/tests.exs @@ -2,6 +2,7 @@ Code.require_file "cases/after_connect_test.exs", __DIR__ Code.require_file "cases/backoff_test.exs", __DIR__ Code.require_file "cases/client_test.exs", __DIR__ Code.require_file "cases/close_test.exs", __DIR__ +Code.require_file "cases/continuation_test.exs", __DIR__ Code.require_file "cases/execute_test.exs", __DIR__ Code.require_file "cases/idle_test.exs", __DIR__ Code.require_file "cases/overflow_test.exs", __DIR__ @@ -9,7 +10,6 @@ Code.require_file "cases/prepare_execute_test.exs", __DIR__ Code.require_file "cases/prepare_stream_test.exs", __DIR__ Code.require_file "cases/prepare_test.exs", __DIR__ Code.require_file "cases/queue_test.exs", __DIR__ -Code.require_file "cases/resource_test.exs", __DIR__ Code.require_file "cases/stage_test.exs", __DIR__ Code.require_file "cases/stream_test.exs", __DIR__ Code.require_file "cases/transaction_execute_test.exs", __DIR__ diff --git a/lib/db_connection.ex b/lib/db_connection.ex index 15b8f37f..1e49f10b 100644 --- a/lib/db_connection.ex +++ b/lib/db_connection.ex @@ -759,10 +759,11 @@ defmodule DBConnection do `run/3` and `transaction/3` can be nested multiple times. If a transaction is rolled back or a nested transaction `fun` raises the transaction is marked as - failed. Any calls inside a failed transaction (except `rollback/2`) will raise - until the outer transaction call returns. All running `transaction/3` calls - will return `{:error, :rollback}` if the transaction failed or connection - closed and `rollback/2` is not called for that `transaction/3`. + failed. Any calls inside a failed transaction (except `rollback/2` and + `transaction/3`) will raise until the outer transaction call returns. All + running (and future) `transaction/3` calls will return `{:error, :rollback}` + if the transaction failed or connection closed and `rollback/2` is not + called for that `transaction/3`. ### Options @@ -820,10 +821,8 @@ defmodule DBConnection do case get_info(conn) do {transaction, _} when transaction in [:transaction, :failed] -> throw({:rollback, conn_ref, err}) - {:idle, _} -> + {idle, _} when idle in [:idle, :continuation] -> raise "not inside transaction" - {:resource, _} -> - raise "not inside resource transaction" :closed -> raise DBConnection.ConnectionError, "connection is closed" end @@ -980,7 +979,8 @@ defmodule DBConnection do case delete_info(conn) do {:idle, conn_state} -> run_checkin(conn, conn_state, opts) - {status, conn_state} when status in [:transaction, :failed, :resource] -> + {status, conn_state} + when status in [:transaction, :failed, :continuation] -> try do raise "inside transaction" catch @@ -996,16 +996,11 @@ defmodule DBConnection do @doc """ Acquire a lock on a connection and return the connection struct for use with - `resource_transaction/3` calls. + `transaction/3` calls. - To use the locked connection call `resource_transaction/3` and run requests - and/or nested run/transactions inside it. If the transaction is rolled back - the connection is checked in. Any calls inside a failed transaction (except - `rollback/2`) will raise until the transaction call returns. All running - `transaction/3` or `resource_transaction/3` calls will return - `{:error, :rollback}` if the transaction failed or connection closed and - `rollback/2` is not called for that `transaction/3` or - `resource_transaction/3`. + To use the locked connection call `transaction/3`. If the transaction is + rolled back the connection is checked in. To release the lock the connection + call `commit_checkin/2` or `rollback_checkin/3`. ### Options @@ -1025,15 +1020,16 @@ defmodule DBConnection do ### Example - conn = DBConnection.resource_begin(pool, opts) - {:ok, res} = DBConnection.resource_transaction(conn, fn(conn) -> + conn = DBConnection.checkout_begin(pool) + {:ok, res} = DBConnection.transaction(conn, fn(conn) -> DBConnection.execute!(conn, "SELECT id FROM table", []) end) + DBConnection.commit_checkin(conn) """ - @spec resource_begin(pool :: GenServer.server, opts :: Keyword.t) :: t - def resource_begin(pool, opts \\ []) do - {result, log_info} = resource_begin_meter(pool, opts) - transaction_log(log_info, :resource_begin) + @spec checkout_begin(pool :: GenServer.server, opts :: Keyword.t) :: t + def checkout_begin(pool, opts \\ []) do + {result, log_info} = checkout_begin_meter(pool, opts) + transaction_log(log_info, :checkout_begin) case result do {:ok, conn} -> conn @@ -1044,69 +1040,14 @@ defmodule DBConnection do end end - @doc """ - Run a series of requests using a connection checked out with - `resource_begin/3`, inside a transaction. - - The result of the transaction fun is return inside an `:ok` tuple: - `{:ok, result}`. If rollback returns `{:error, reason}` or if connection - not available `{:error, :rollback}`. - - To use the locked connection call the request with the connection - reference passed as the single argument to the `fun`. If the - connection disconnects all future calls using that connection - reference will fail. - - `run/3`, `transaction/3` and `resource_transaction/3` can be nested inside - `resource_transaction/3`. If a resource transaction is rolled back or `fun` - raises the transaction is rolled back and the connection is checked in. If - `resource_transaction/3` is nested in a `resource_transaction/3` or - `transaction/3` it behaves like `transaction/3`. - - ### Options - - * `:log` - A function to log information about begin, commit and rollback - calls made as part of the transaction, either a 1-arity fun, - `{module, function, args}` with `DBConnection.LogEntry.t` prepended to - `args` or `nil`. See `DBConnection.LogEntry` (default: `nil`) - - The pool and connection module may support other options. All options - are passed to `handle_rollback/2`. - - ### Example - - conn = DBConnection.resource_begin(pool, opts) - {:ok, res} = DBConnection.resource_transaction(conn, fn(conn) -> - DBConnection.execute!(conn, "SELECT id FROM table", []) - end) - DBConnection.resource_commit(conn, opts) - - """ - @spec resource_transaction(t, (t -> result), opts :: Keyword.t) :: - {:ok, result} | {:error, reason :: any | :rollback} when result: var - def resource_transaction(conn, fun, opts \\ []) do - case get_info(conn) do - {:failed, _} -> - {:error, :rollback} - {:idle, _} -> - raise "not inside transaction" - {:transaction, _} -> - transaction_nested(conn, fun) - {:resource, conn_state} -> - resource_nested(conn, conn_state, fun, opts) - :closed -> - {:error, :rollback} - end - end - @doc """ Commit transaction and release the lock on a connection. Returns `:ok` on sucess, otherwise `{:error, :rollback}` if the transaction failed and was rolled back or the connection is not available. - Can only be called for a connection checked out with `resource_begin/2` when - outside of `resource_transaction/3`. + Can only be called for a connection checked out with `checkout_begin/2` when + outside of `transaction/3`. ### Options @@ -1120,14 +1061,14 @@ defmodule DBConnection do ### Example - conn = DBConnection.resource_begin(pool, opts) - DBConnection.resource_commit(conn, opts) + conn = DBConnection.checkout_begin(pool) + DBConnection.commit_checkin(conn) """ - @spec resource_commit(t, Keyword.t) :: :ok | {:error, :rollback} - def resource_commit(conn, opts \\ []) do + @spec commit_checkin(t, Keyword.t) :: :ok | {:error, :rollback} + def commit_checkin(conn, opts \\ []) do log = Keyword.get(opts, :log) - {result, log_info} = resource_conclude(conn, &commit/4, log, opts, :ok) - transaction_log(log_info, :resource_commit) + {result, log_info} = continuation_conclude(conn, &commit/4, log, opts, :ok) + transaction_log(log_info, :commit_checkin) case result do {:raise, err} -> raise err @@ -1143,8 +1084,8 @@ defmodule DBConnection do Returns `{:error, reason}` if rolls back or connection not available. - Can only be called for a connection checked out with `resource_begin/2` when - outside of `resource_transaction/3`. + Can only be called for a connection checked out with `checkout_begin/2` when + outside of `transaction/3`. ### Options @@ -1158,14 +1099,14 @@ defmodule DBConnection do ### Example - conn = DBConnection.resource_begin(pool, opts) - DBConnection.resource_rollback(conn, reason, opts) + conn = DBConnection.checkout_begin(pool) + DBConnection.rollback_checkin(conn, :oops) """ - @spec resource_rollback(t, reason, Keyword.t) :: + @spec rollback_checkin(t, reason, Keyword.t) :: {:error, reason} when reason: var - def resource_rollback(conn, reason, opts \\ []) do - {result, log_info} = run_resource_rollback(conn, reason, opts) - transaction_log(log_info, :resource_rollback) + def rollback_checkin(conn, reason, opts \\ []) do + {result, log_info} = run_continuation_rollback(conn, reason, opts) + transaction_log(log_info, :rollback_checkin) case result do {:raise, err} -> raise err @@ -1461,12 +1402,18 @@ defmodule DBConnection do defp log_result(other), do: other defp transaction_meter(%DBConnection{} = conn, fun, opts) do - case fetch_info(conn) do + case get_info(conn) do {:transaction, _} -> {transaction_nested(conn, fun), nil} + {:continuation, conn_state} -> + {transaction_continue(conn, conn_state, fun, opts), nil} {:idle, conn_state} -> log = Keyword.get(opts, :log) begin_meter(conn, conn_state, log, [], fun, opts) + {:failed, _} -> + {{:error, :rollback}, nil} + :closed -> + {{:error, :rollback}, nil} end end defp transaction_meter(pool, fun, opts) do @@ -1529,7 +1476,7 @@ defmodule DBConnection do defp commit(conn, log, opts, result) do case get_info(conn) do - {trans, conn_state} when trans in [:transaction, :resource] -> + {trans, conn_state} when trans in [:transaction, :continuation] -> conclude_meter(conn, conn_state, log, :handle_commit, opts, result) {:failed, conn_state} -> result = {:error, :rollback} @@ -1541,7 +1488,8 @@ defmodule DBConnection do defp rollback(conn, log, opts, result) do case get_info(conn) do - {trans, conn_state} when trans in [:transaction, :failed, :resource] -> + {trans, conn_state} + when trans in [:transaction, :failed, :continuation] -> conclude_meter(conn, conn_state, log, :handle_rollback, opts, result) :closed -> {result, nil} @@ -1636,6 +1584,60 @@ defmodule DBConnection do end end + defp transaction_continue(conn, state, fun, opts) do + %DBConnection{conn_ref: conn_ref} = conn + try do + put_info(conn, :transaction, state) + fun.(conn) + catch + :throw, {:rollback, ^conn_ref, reason} -> + continue_failed(conn, reason, opts) + kind, reason -> + stack = System.stacktrace() + continue_failed(conn, :raise, opts) + :erlang.raise(kind, reason, stack) + else + result -> + continue_ok(conn, result, opts) + end + end + + defp continue_ok(conn, result, opts) do + case get_info(conn) do + {:transaction, conn_state} -> + put_info(conn, :continuation, conn_state) + {:ok, result} + {:failed, conn_state} -> + put_info(conn, :continuation, conn_state) + continue_rollback(conn, :rollback, opts) + _ -> + {:error, :rollback} + end + end + + defp continue_failed(conn, reason, opts) do + case get_info(conn) do + {trans, conn_state} when trans in [:transaction, :failed] -> + put_info(conn, :continuation, conn_state) + continue_rollback(conn, reason, opts) + :closed -> + {:error, reason} + end + end + + defp continue_rollback(conn, reason, opts) do + {result, log_info} = run_continuation_rollback(conn, reason, opts) + transaction_log(log_info) + case result do + {:raise, err} -> + raise err + {kind, reason, stack} -> + :erlang.raise(kind, reason, stack) + other -> + other + end + end + @doc false def prepare_declare(conn, query, params, opts) do query = parse(:prepare_declare, query, params, opts) @@ -1776,27 +1778,27 @@ defmodule DBConnection do Stream.resource(start, next, stop) end - defp resource_begin_meter(pool, opts) do + defp checkout_begin_meter(pool, opts) do case Keyword.get(opts, :log) do nil -> - resource_begin_meter(pool, nil, [], opts) + checkout_begin_meter(pool, nil, [], opts) log -> times = [checkout: time()] - resource_begin_meter(pool, log, times, opts) + checkout_begin_meter(pool, log, times, opts) end end - defp resource_begin_meter(pool, log, times, opts) do + defp checkout_begin_meter(pool, log, times, opts) do case run_checkout(pool, opts) do {:ok, conn, conn_state} -> - resource_begin_meter(conn, conn_state, log, times, opts) + checkout_begin_meter(conn, conn_state, log, times, opts) {:error, err} -> {:raise, err} end end - defp resource_begin_meter(conn, conn_state, nil, [], opts) do - case handle(conn, conn_state, :handle_begin, opts, :resource) do + defp checkout_begin_meter(conn, conn_state, nil, [], opts) do + case handle(conn, conn_state, :handle_begin, opts, :continuation) do {:ok, _} -> {{:ok, conn}, nil} error -> @@ -1804,96 +1806,42 @@ defmodule DBConnection do {error, nil} end end - defp resource_begin_meter(conn, conn_state, log, times, opts) do + defp checkout_begin_meter(conn, conn_state, log, times, opts) do start = time() - result = handle(conn, conn_state, :handle_begin, opts, :resource) + result = handle(conn, conn_state, :handle_begin, opts, :continuation) times = [stop: time(), start: start] ++ times log_info = {log, times, :handle_begin, result} case result do {:ok, _} -> - resource_begin_log(conn, log_info, opts) + checkout_begin_log(conn, log_info, opts) error -> checkin(conn, opts) {error, log_info} end end - defp resource_begin_log(conn, {log, _, _, _} = log_info, opts) do + defp checkout_begin_log(conn, {log, _, _, _} = log_info, opts) do try do - transaction_log(log_info, :resource_begin) + transaction_log(log_info, :checkout_begin) catch kind, reason -> result = {kind, reason, System.stacktrace()} - resource_conclude(conn, &rollback/4, log, opts, result) + continuation_conclude(conn, &rollback/4, log, opts, result) else _ -> {{:ok, conn}, nil} end end - defp resource_nested(conn, state, fun, opts) do - %DBConnection{conn_ref: conn_ref} = conn - try do - put_info(conn, :transaction, state) - fun.(conn) - catch - :throw, {:rollback, ^conn_ref, reason} -> - resource_failed(conn, reason, opts) - kind, reason -> - stack = System.stacktrace() - resource_failed(conn, :raise, opts) - :erlang.raise(kind, reason, stack) - else - result -> - resource_ok(conn, result, opts) - end - end - - defp resource_ok(conn, result, opts) do - case get_info(conn) do - {:transaction, conn_state} -> - put_info(conn, :resource, conn_state) - {:ok, result} - {:failed, conn_state} -> - put_info(conn, :resource, conn_state) - resource_nested_rollback(conn, :rollback, opts) - _ -> - {:error, :rollback} - end - end - - defp resource_failed(conn, reason, opts) do - case get_info(conn) do - {trans, conn_state} when trans in [:transaction, :failed] -> - put_info(conn, :resource, conn_state) - resource_nested_rollback(conn, reason, opts) - :closed -> - {:error, reason} - end - end - - defp resource_nested_rollback(conn, reason, opts) do - {result, log_info} = run_resource_rollback(conn, reason, opts) - transaction_log(log_info, :resource_transaction) - case result do - {:raise, err} -> - raise err - {kind, reason, stack} -> - :erlang.raise(kind, reason, stack) - other -> - other - end - end - - defp run_resource_rollback(conn, reason, opts) do + defp run_continuation_rollback(conn, reason, opts) do log = Keyword.get(opts, :log) result = {:error, reason} - resource_conclude(conn, &rollback/4, log, opts, result, result) + continuation_conclude(conn, &rollback/4, log, opts, result, result) end - defp resource_conclude(conn, fun, log, opts, result, closed \\ {:error, :rollback}) do + defp continuation_conclude(conn, fun, log, opts, result, closed \\ {:error, :rollback}) do case get_info(conn) do - {:resource, _} -> + {:continuation, _} -> try do fun.(conn, log, opts, result) after @@ -1915,8 +1863,8 @@ defmodule DBConnection do case get_info(conn) do {:failed, _} -> raise DBConnection.ConnectionError, "transaction rolling back" - {:resource, _} -> - raise "not inside resource transaction" + {:continuation, _} -> + raise "not inside transaction" {_, _} = info -> info :closed -> diff --git a/lib/db_connection/stage.ex b/lib/db_connection/stage.ex index c56976b7..9bffae7e 100644 --- a/lib/db_connection/stage.ex +++ b/lib/db_connection/stage.ex @@ -100,11 +100,11 @@ defmodule DBConnection.Stage do def terminate(reason, %Stage{transaction?: true} = stage) do %Stage{conn: conn, state: state, opts: opts} = stage deallocate = &deallocate(&1, reason, state, opts) - case DBConnection.resource_transaction(conn, deallocate, opts) do + case DBConnection.transaction(conn, deallocate, opts) do {:ok, :normal} -> - DBConnection.resource_commit(conn, opts) + DBConnection.commit_checkin(conn, opts) {:ok, reason} -> - DBConnection.resource_rollback(conn, reason, opts) + DBConnection.rollback_checkin(conn, reason, opts) {:error, :rollback} -> :ok end @@ -123,7 +123,7 @@ defmodule DBConnection.Stage do defp init(pool, opts) do case Keyword.get(opts, :stage_transaction, true) do true -> - conn = DBConnection.resource_begin(pool, opts) + conn = DBConnection.checkout_begin(pool, opts) %Stage{conn: conn, transaction?: true, state: :declare, opts: opts} false -> conn = DBConnection.checkout(pool, opts) @@ -132,7 +132,7 @@ defmodule DBConnection.Stage do end defp run(fun, opts, %Stage{conn: conn, transaction?: true}) do - case DBConnection.resource_transaction(conn, fun, opts) do + case DBConnection.transaction(conn, fun, opts) do {:ok, result} -> result {:error, reason} -> diff --git a/test/test_support.exs b/test/test_support.exs index 7fd7c796..55c362d3 100644 --- a/test/test_support.exs +++ b/test/test_support.exs @@ -67,20 +67,16 @@ defmodule TestConnection do DBConnection.Stage.start_link(pool, query, params, opts2 ++ unquote(opts)) end - def resource_begin(pool, opts2 \\ []) do - DBConnection.resource_begin(pool, opts2 ++ unquote(opts)) + def checkout_begin(pool, opts2 \\ []) do + DBConnection.checkout_begin(pool, opts2 ++ unquote(opts)) end - def resource_transaction(conn, fun, opts2 \\ []) do - DBConnection.resource_transaction(conn, fun, opts2 ++ unquote(opts)) + def commit_checkin(conn, opts2 \\ []) do + DBConnection.commit_checkin(conn, opts2 ++ unquote(opts)) end - def resource_commit(conn, opts2 \\ []) do - DBConnection.resource_commit(conn, opts2 ++ unquote(opts)) - end - - def resource_rollback(conn, reason, opts2 \\ []) do - DBConnection.resource_rollback(conn, reason, opts2 ++ unquote(opts)) + def rollback_checkin(conn, reason, opts2 \\ []) do + DBConnection.rollback_checkin(conn, reason, opts2 ++ unquote(opts)) end def checkin(conn, opts2 \\ []) do From 802a3019a003447481b2a6d86acb0b838db45747 Mon Sep 17 00:00:00 2001 From: James Fish Date: Sun, 28 May 2017 13:25:28 +0100 Subject: [PATCH 15/20] Log pool checkout errors --- integration_test/cases/queue_test.exs | 101 +++++++++++++++++++++++++- lib/db_connection.ex | 55 ++++++++++---- lib/db_connection/log_entry.ex | 3 + 3 files changed, 144 insertions(+), 15 deletions(-) diff --git a/integration_test/cases/queue_test.exs b/integration_test/cases/queue_test.exs index 0383ace6..6f5194dc 100644 --- a/integration_test/cases/queue_test.exs +++ b/integration_test/cases/queue_test.exs @@ -3,8 +3,9 @@ defmodule QueueTest do alias TestPool, as: P alias TestAgent, as: A + alias TestQuery, as: Q - test "queue: false raises on busy" do + test "run queue: false raises on busy" do stack = [{:ok, :state}] {:ok, agent} = A.start_link(stack) @@ -22,6 +23,104 @@ defmodule QueueTest do end) end + test "execute queue: false raises on busy" do + stack = [{:ok, :state}] + {:ok, agent} = A.start_link(stack) + + parent = self() + opts = [agent: agent, parent: parent] + {:ok, pool} = P.start_link(opts) + + P.run(pool, fn(_) -> + {queue_time, _} = :timer.tc(fn() -> + opts = [queue: false] ++ opts + assert_raise DBConnection.ConnectionError, + "connection not available and queuing is disabled", + fn() -> P.execute!(pool, %Q{}, [:param], opts) end + end) + assert queue_time <= 1_000_000, "request was queued" + end) + end + + test "execute queue: false raises on busy and logs" do + stack = [{:ok, :state}] + {:ok, agent} = A.start_link(stack) + + parent = self() + opts = [agent: agent, parent: parent] + {:ok, pool} = P.start_link(opts) + + P.run(pool, fn(_) -> + {queue_time, _} = :timer.tc(fn() -> + log = &send(parent, &1) + opts = [queue: false, log: log] ++ opts + assert_raise DBConnection.ConnectionError, + "connection not available and queuing is disabled", + fn() -> P.execute!(pool, %Q{}, [:param], opts) end + end) + assert queue_time <= 1_000_000, "request was queued" + end) + + assert_received %DBConnection.LogEntry{call: :execute} = entry + assert %{query: %Q{}, params: [:param], result: result} = entry + assert {:error, %DBConnection.ConnectionError{}} = result + assert is_integer(entry.pool_time) + assert entry.pool_time >= 0 + assert is_nil(entry.connection_time) + assert is_nil(entry.decode_time) + end + + test "transaction queue: false raises on busy" do + stack = [{:ok, :state}] + {:ok, agent} = A.start_link(stack) + + parent = self() + opts = [agent: agent, parent: parent] + {:ok, pool} = P.start_link(opts) + + P.run(pool, fn(_) -> + {queue_time, _} = :timer.tc(fn() -> + opts = [queue: false] ++ opts + assert_raise DBConnection.ConnectionError, + "connection not available and queuing is disabled", + fn() -> + P.transaction(pool, fn() -> flunk("should not run") end, opts) + end + end) + assert queue_time <= 1_000_000, "request was queued" + end) + end + + test "transaction queue: false raises on busy and logs" do + stack = [{:ok, :state}] + {:ok, agent} = A.start_link(stack) + + parent = self() + opts = [agent: agent, parent: parent] + {:ok, pool} = P.start_link(opts) + + P.run(pool, fn(_) -> + {queue_time, _} = :timer.tc(fn() -> + log = &send(parent, &1) + opts = [queue: false, log: log] ++ opts + assert_raise DBConnection.ConnectionError, + "connection not available and queuing is disabled", + fn() -> + P.transaction(pool, fn() -> flunk("should not run") end, opts) + end + end) + assert queue_time <= 1_000_000, "request was queued" + end) + + assert_received %DBConnection.LogEntry{call: :transaction} = entry + assert %{query: :begin, params: nil, result: result} = entry + assert {:error, %DBConnection.ConnectionError{}} = result + assert is_integer(entry.pool_time) + assert entry.pool_time >= 0 + assert is_nil(entry.connection_time) + assert is_nil(entry.decode_time) + end + test "queue many async" do stack = [{:ok, :state}] {:ok, agent} = A.start_link(stack) diff --git a/lib/db_connection.ex b/lib/db_connection.ex index 1e49f10b..1e8c2bd6 100644 --- a/lib/db_connection.ex +++ b/lib/db_connection.ex @@ -1327,29 +1327,40 @@ defmodule DBConnection do defp run_meter(%DBConnection{} = conn, fun, opts) do case Keyword.get(opts, :log) do nil -> - {run(conn, fun, opts), nil} + {fun.(conn), nil} log -> - run_meter(conn, log, [], fun, opts) - end + start = time() + result = fun.(conn) + stop = time() + meter = {log, [stop: stop, start: start]} + {result, meter} + end end defp run_meter(pool, fun, opts) do case Keyword.get(opts, :log) do nil -> {run(pool, fun, opts), nil} log -> - run_meter(pool, log, [checkout: time()], fun, opts) + run_meter(pool, log, fun, opts) end end - defp run_meter(conn, log, times, fun, opts) do - fun = fn(conn2) -> - start = time() - result = fun.(conn2) - stop = time() - meter = {log, [stop: stop, start: start] ++ times} - {result, meter} + defp run_meter(pool, log, fun, opts) do + checkout = time() + case run_checkout(pool, opts) do + {:ok, conn, _} -> + try do + start = time() + result = fun.(conn) + stop = time() + meter = {log, [stop: stop, start: start, checkout: checkout]} + {result, meter} + after + checkin(conn, opts) + end + {:error, _} = error -> + {error, {log, [stop: time(), checkout: checkout]}} end - run(conn, fun, opts) end defp decode_log(_, _, _, nil, result), do: log_result(result) @@ -1421,8 +1432,24 @@ defmodule DBConnection do nil -> run(pool, &begin(&1, nil, [], fun, opts), opts) log -> - times = [checkout: time()] - run(pool, &begin(&1, log, times, fun, opts), opts) + transaction_meter(pool, log, fun, opts) + end + end + + defp transaction_meter(pool, log, fun, opts) do + checkout = time() + case run_checkout(pool, opts) do + {:ok, conn, conn_state} -> + try do + begin_meter(conn, conn_state, log, [checkout: checkout], fun, opts) + after + checkin(conn, opts) + end + {:error, err} -> + times = [stop: time(), checkout: checkout] + result = {:raise, err} + log_info = {log, times, :handle_begin, result} + {result, log_info} end end diff --git a/lib/db_connection/log_entry.ex b/lib/db_connection/log_entry.ex index a5356dec..131092fc 100644 --- a/lib/db_connection/log_entry.ex +++ b/lib/db_connection/log_entry.ex @@ -62,4 +62,7 @@ defmodule DBConnection.LogEntry do defp parse_time({:checkout, checkout} = time, {{:start, start}, entry}) do {time, %{entry | pool_time: diff(start, checkout)}} end + defp parse_time({:checkout, checkout} = time, {{:stop, stop}, entry}) do + {time, %{entry | pool_time: diff(stop, checkout)}} + end end From 5cc2f7496d8da2440c7af224d0891a6bdaede5d0 Mon Sep 17 00:00:00 2001 From: James Fish Date: Sun, 28 May 2017 13:51:07 +0100 Subject: [PATCH 16/20] Require elixir 1.3+ --- .travis.yml | 6 +----- mix.exs | 2 +- 2 files changed, 2 insertions(+), 6 deletions(-) diff --git a/.travis.yml b/.travis.yml index f568d758..84a820e9 100644 --- a/.travis.yml +++ b/.travis.yml @@ -2,16 +2,12 @@ language: elixir env: - ELIXIR_ERL_OPTIONS="+T 9" elixir: - - 1.2.6 - 1.3.0 - 1.3.4 otp_release: - 18.3 - 19.1 -matrix: - include: - - elixir: 1.2.0 - otp_release: 18.2 + - 19.3 sudo: false script: - mix test.all diff --git a/mix.exs b/mix.exs index 3ea9fd58..4259fae4 100644 --- a/mix.exs +++ b/mix.exs @@ -7,7 +7,7 @@ defmodule DBConnection.Mixfile do def project do [app: :db_connection, version: @version, - elixir: "~> 1.2", + elixir: "~> 1.3", deps: deps(), docs: docs(), description: description(), From 7d298c9f3a4b2a72f76d4aeff5649e26bb400fa6 Mon Sep 17 00:00:00 2001 From: James Fish Date: Sun, 28 May 2017 17:30:07 +0100 Subject: [PATCH 17/20] Rename stream_map -> stream_mapper --- integration_test/cases/stage_test.exs | 34 +++++++++++++++++ integration_test/cases/stream_test.exs | 37 ++++++++++++++++++ .../cases/transaction_stage_test.exs | 38 +++++++++++++++++++ lib/db_connection.ex | 16 ++++---- lib/db_connection/stage.ex | 13 ++++--- 5 files changed, 124 insertions(+), 14 deletions(-) diff --git a/integration_test/cases/stage_test.exs b/integration_test/cases/stage_test.exs index 5fb0557c..199212da 100644 --- a/integration_test/cases/stage_test.exs +++ b/integration_test/cases/stage_test.exs @@ -95,6 +95,40 @@ defmodule StageTest do ] = A.record(agent) end + test "stage with execute in stream_mapper" do + stack = [ + {:ok, :state}, + {:ok, %C{}, :new_state}, + {:ok, %R{}, :newer_state}, + {:ok, %R{}, :newest_state}, + {:deallocate, %R{}, :state2}, + {:ok, %R{}, :new_state2}, + {:ok, :deallocated, :newer_state2}, + ] + {:ok, agent} = A.start_link(stack) + + opts = [agent: agent, parent: self(), stage_transaction: false] + {:ok, pool} = P.start_link(opts) + + mapper = fn(conn, res) -> [P.execute!(conn, %Q{}, res, opts), :mapped] end + opts = [stream_mapper: mapper] ++ opts + {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) + mon = Process.monitor(stage) + assert [{stage, [cancel: :transient]}] |> GenStage.stream() |> Enum.to_list() == [%R{}, :mapped, %R{}, :mapped] + + assert_receive {:DOWN, ^mon, :process, ^stage, :normal} + + assert [ + connect: [_], + handle_declare: [%Q{}, [:param], _, :state], + handle_first: [%Q{}, %C{}, _, :new_state], + handle_execute: [%Q{}, %R{}, _, :newer_state], + handle_next: [%Q{}, %C{}, _, :newest_state], + handle_execute: [%Q{}, %R{}, _, :state2], + handle_deallocate: [%Q{}, %C{}, _, :new_state2], + ] = A.record(agent) + end + test "stage checks in on abnormal exit" do stack = [ {:ok, :state}, diff --git a/integration_test/cases/stream_test.exs b/integration_test/cases/stream_test.exs index 4bb92e43..b2965e56 100644 --- a/integration_test/cases/stream_test.exs +++ b/integration_test/cases/stream_test.exs @@ -71,6 +71,43 @@ defmodule StreamTest do ] = A.record(agent) end + test "stream with execute in stream_mapper" do + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, %C{}, :newer_state}, + {:ok, %R{}, :newest_state}, + {:ok, %R{}, :state2}, + {:deallocate, %R{}, :new_state2}, + {:ok, %R{}, :newer_state2}, + {:ok, :deallocated, :newest_state2}, + {:ok, :commited, :state3} + ] + {:ok, agent} = A.start_link(stack) + + opts = [agent: agent, parent: self()] + {:ok, pool} = P.start_link(opts) + assert P.transaction(pool, fn(conn) -> + mapper = fn(conn, res) -> [P.execute!(conn, %Q{}, res, opts), :mapped] end + stream = P.stream(conn, %Q{}, [:param], [stream_mapper: mapper]) + assert %DBConnection.Stream{} = stream + assert Enum.to_list(stream) == [%R{}, :mapped, %R{}, :mapped] + :hi + end) == {:ok, :hi} + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_declare: [%Q{}, [:param], _, :new_state], + handle_first: [%Q{}, %C{}, _, :newer_state], + handle_execute: [%Q{}, %R{}, _, :newest_state], + handle_next: [%Q{}, %C{}, _, :state2], + handle_execute: [%Q{}, %R{}, _, :new_state2], + handle_deallocate: [%Q{}, %C{}, _, :newer_state2], + handle_commit: [_, :newest_state2] + ] = A.record(agent) + end + test "stream logs result" do stack = [ {:ok, :state}, diff --git a/integration_test/cases/transaction_stage_test.exs b/integration_test/cases/transaction_stage_test.exs index ae770dc1..bb181a28 100644 --- a/integration_test/cases/transaction_stage_test.exs +++ b/integration_test/cases/transaction_stage_test.exs @@ -107,6 +107,44 @@ defmodule TransactionStageTest do ] = A.record(agent) end + test "stage with execute in stream_mapper" do + stack = [ + {:ok, :state}, + {:ok, :began, :new_state}, + {:ok, %C{}, :newer_state}, + {:ok, %R{}, :newest_state}, + {:ok, %R{}, :state2}, + {:deallocate, %R{}, :new_state2}, + {:ok, %R{}, :newer_state2}, + {:ok, :deallocated, :newest_state2}, + {:ok, :committed, :state3} + ] + {:ok, agent} = A.start_link(stack) + + opts = [agent: agent, parent: self()] + {:ok, pool} = P.start_link(opts) + + mapper = fn(conn, res) -> [P.execute!(conn, %Q{}, res, opts), :mapped] end + opts = [stream_mapper: mapper] ++ opts + {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) + mon = Process.monitor(stage) + assert [{stage, [cancel: :transient]}] |> GenStage.stream() |> Enum.to_list() == [%R{}, :mapped, %R{}, :mapped] + + assert_receive {:DOWN, ^mon, :process, ^stage, :normal} + + assert [ + connect: [_], + handle_begin: [_, :state], + handle_declare: [%Q{}, [:param], _, :new_state], + handle_first: [%Q{}, %C{}, _, :newer_state], + handle_execute: [%Q{}, %R{}, _, :newest_state], + handle_next: [%Q{}, %C{}, _, :state2], + handle_execute: [%Q{}, %R{}, _, :new_state2], + handle_deallocate: [%Q{}, %C{}, _, :newer_state2], + handle_commit: [_, :newest_state2] + ] = A.record(agent) + end + test "stage rolls back on abnormal exit" do stack = [ {:ok, :state}, diff --git a/lib/db_connection.ex b/lib/db_connection.ex index 1e8c2bd6..9325078b 100644 --- a/lib/db_connection.ex +++ b/lib/db_connection.ex @@ -834,9 +834,9 @@ defmodule DBConnection do ### Options - * `:stream_map` - A function to flat map the results of the query, either a - 2-arity fun, `{module, function, args}` with `DBConnection.t` and the result - prepended to `args` or `nil` (default: `nil` + * `:stream_mapper` - A function to flat map the results of the query, either + a 2-arity fun, `{module, function, args}` with `DBConnection.t` and the + result prepended to `args` or `nil` (default: `nil`) * `:pool_timeout` - The maximum time to wait for a reply when making a synchronous call to the pool (default: `5_000`) * `:queue` - Whether to block waiting in an internal queue for the @@ -873,9 +873,9 @@ defmodule DBConnection do ### Options - * `:stream_map` - A function to flat map the results of the query, either a - 2-arity fun, `{module, function, args}` with `DBConnection.t` and the result - prepended to `args` or `nil` (default: `nil`) + * `:stream_mapper` - A function to flat map the results of the query, + either a 2-arity fun, `{module, function, args}` with `DBConnection.t` and + the result prepended to `args` or `nil` (default: `nil`) * `:pool_timeout` - The maximum time to wait for a reply when making a synchronous call to the pool (default: `5_000`) * `:queue` - Whether to block waiting in an internal queue for the @@ -896,7 +896,7 @@ defmodule DBConnection do {:ok, results} = DBConnection.transaction(conn, fn(conn) -> query = %Query{statement: "SELECT id FROM table"} query = DBConnection.prepare!(conn, query) - opts = [stream_map: &Map.fetch!(&1, :rows)] + opts = [stream_mapper: &Map.fetch!(&1, :rows)] stream = DBConnection.stream(conn, query, [], opts) Enum.to_list(stream) end) @@ -1771,7 +1771,7 @@ defmodule DBConnection do end defp fetch_map(conn, result, opts) do - case Keyword.get(opts, :stream_map) do + case Keyword.get(opts, :stream_mapper) do map when is_function(map, 2) -> map.(conn, result) {mod, fun, args} -> diff --git a/lib/db_connection/stage.ex b/lib/db_connection/stage.ex index 9bffae7e..9e2e6416 100644 --- a/lib/db_connection/stage.ex +++ b/lib/db_connection/stage.ex @@ -5,9 +5,9 @@ defmodule DBConnection.Stage do ### Options - * `:stream_map` - A function to flat map the results of the query, either a - 2-arity fun, `{module, function, args}` with `DBConnection.t` and the result - prepended to `args` or `nil` (default: `nil`) + * `:stream_mapper` - A function to flat map the results of the query, either + a 2-arity fun, `{module, function, args}` with `DBConnection.t` and the + result prepended to `args` or `nil` (default: `nil`) * `:stage_prepare` - Whether the producer should prepare the query before streaming it (default: `false`) * `:stage_transaction` - Whether the producer should encapsulate the query @@ -51,7 +51,7 @@ defmodule DBConnection.Stage do ### Example query = %Query{statement: "SELECT id FROM table"} - opts = [stream_map: &Map.fetch!(&1, :rows)] + opts = [stream_mapper: &Map.fetch!(&1, :rows)] {:ok, stage} = DBConnection.Stage.start_link(pool, query, [], opts) stage |> GenStage.stream() |> Enum.to_list() """ @@ -88,8 +88,9 @@ defmodule DBConnection.Stage do GenStage.async_info(self(), :stop) {:noreply, [], %Stage{stage | state: state}} {events, state} -> - # stream_map may produce the desired number of events, i.e. at the end - # of the results so we can close the cursor as soon as possible. + # stream_mapper may not produce the desired number of events, i.e. at + # the end of the results, so we can close the cursor as soon as + # possible. pending = demand - length(events) _ = if pending > 0, do: send(self(), {:fetch, conn, pending}) {:noreply, events, %Stage{stage | state: state}} From 37a505ae717206fc07188f1412736ec19f4373e0 Mon Sep 17 00:00:00 2001 From: James Fish Date: Sun, 28 May 2017 17:34:13 +0100 Subject: [PATCH 18/20] Fix checkout_begin pool error handling --- integration_test/cases/queue_test.exs | 47 +++++++++++++++++++++++++++ lib/db_connection.ex | 27 +++++++++++---- 2 files changed, 67 insertions(+), 7 deletions(-) diff --git a/integration_test/cases/queue_test.exs b/integration_test/cases/queue_test.exs index 6f5194dc..78ea4e72 100644 --- a/integration_test/cases/queue_test.exs +++ b/integration_test/cases/queue_test.exs @@ -121,6 +121,53 @@ defmodule QueueTest do assert is_nil(entry.decode_time) end + test "checkout_begin queue: false raises on busy" do + stack = [{:ok, :state}] + {:ok, agent} = A.start_link(stack) + + parent = self() + opts = [agent: agent, parent: parent] + {:ok, pool} = P.start_link(opts) + + P.run(pool, fn(_) -> + {queue_time, _} = :timer.tc(fn() -> + opts = [queue: false] ++ opts + assert_raise DBConnection.ConnectionError, + "connection not available and queuing is disabled", + fn() -> P.checkout_begin(pool, opts) end + end) + assert queue_time <= 1_000_000, "request was queued" + end) + end + + test "checkout_begin queue: false raises on busy and logs" do + stack = [{:ok, :state}] + {:ok, agent} = A.start_link(stack) + + parent = self() + opts = [agent: agent, parent: parent] + {:ok, pool} = P.start_link(opts) + + P.run(pool, fn(_) -> + {queue_time, _} = :timer.tc(fn() -> + log = &send(parent, &1) + opts = [queue: false, log: log] ++ opts + assert_raise DBConnection.ConnectionError, + "connection not available and queuing is disabled", + fn() -> P.checkout_begin(pool, opts) end + end) + assert queue_time <= 1_000_000, "request was queued" + end) + + assert_received %DBConnection.LogEntry{call: :checkout_begin} = entry + assert %{query: :begin, params: nil, result: result} = entry + assert {:error, %DBConnection.ConnectionError{}} = result + assert is_integer(entry.pool_time) + assert entry.pool_time >= 0 + assert is_nil(entry.connection_time) + assert is_nil(entry.decode_time) + end + test "queue many async" do stack = [{:ok, :state}] {:ok, agent} = A.start_link(stack) diff --git a/lib/db_connection.ex b/lib/db_connection.ex index 9325078b..0118b3f5 100644 --- a/lib/db_connection.ex +++ b/lib/db_connection.ex @@ -1808,23 +1808,22 @@ defmodule DBConnection do defp checkout_begin_meter(pool, opts) do case Keyword.get(opts, :log) do nil -> - checkout_begin_meter(pool, nil, [], opts) + run_checkout_begin(pool, opts) log -> - times = [checkout: time()] - checkout_begin_meter(pool, log, times, opts) + checkout_begin_meter(pool, log, opts) end end - defp checkout_begin_meter(pool, log, times, opts) do + defp run_checkout_begin(pool, opts) do case run_checkout(pool, opts) do {:ok, conn, conn_state} -> - checkout_begin_meter(conn, conn_state, log, times, opts) + run_checkout_begin(conn, conn_state, opts) {:error, err} -> - {:raise, err} + {{:raise, err}, nil} end end - defp checkout_begin_meter(conn, conn_state, nil, [], opts) do + defp run_checkout_begin(conn, conn_state, opts) do case handle(conn, conn_state, :handle_begin, opts, :continuation) do {:ok, _} -> {{:ok, conn}, nil} @@ -1833,6 +1832,20 @@ defmodule DBConnection do {error, nil} end end + + defp checkout_begin_meter(pool, log, opts) do + checkout = time() + case run_checkout(pool, opts) do + {:ok, conn, conn_state} -> + checkout_begin_meter(conn, conn_state, log, [checkout: checkout], opts) + {:error, err} -> + times = [stop: time(), checkout: checkout] + result = {:raise, err} + log_info = {log, times, :handle_begin, result} + {result, log_info} + end + end + defp checkout_begin_meter(conn, conn_state, log, times, opts) do start = time() result = handle(conn, conn_state, :handle_begin, opts, :continuation) From 9c6e4945112554bf40c240244c033f6dcc9c4a13 Mon Sep 17 00:00:00 2001 From: James Fish Date: Sun, 28 May 2017 17:40:00 +0100 Subject: [PATCH 19/20] Rename Stage -> Producer --- .../{stage_test.exs => producer_test.exs} | 24 +++++++------- ...test.exs => transaction_producer_test.exs} | 24 +++++++------- integration_test/tests.exs | 4 +-- lib/db_connection/{stage.ex => producer.ex} | 32 +++++++++---------- test/test_support.exs | 4 +-- 5 files changed, 44 insertions(+), 44 deletions(-) rename integration_test/cases/{stage_test.exs => producer_test.exs} (93%) rename integration_test/cases/{transaction_stage_test.exs => transaction_producer_test.exs} (93%) rename lib/db_connection/{stage.ex => producer.ex} (83%) diff --git a/integration_test/cases/stage_test.exs b/integration_test/cases/producer_test.exs similarity index 93% rename from integration_test/cases/stage_test.exs rename to integration_test/cases/producer_test.exs index 199212da..9244b94e 100644 --- a/integration_test/cases/stage_test.exs +++ b/integration_test/cases/producer_test.exs @@ -1,4 +1,4 @@ -defmodule StageTest do +defmodule ProducerTest do use ExUnit.Case, async: true alias TestPool, as: P @@ -19,7 +19,7 @@ defmodule StageTest do opts = [agent: agent, parent: self(), stage_transaction: false] {:ok, pool} = P.start_link(opts) - {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) + {:ok, stage} = P.start_producer(pool, %Q{}, [:param], opts) mon = Process.monitor(stage) assert [{stage, [cancel: :transient]}] |> GenStage.stream() |> Enum.to_list() == [%R{}, %R{}] @@ -47,7 +47,7 @@ defmodule StageTest do opts = [agent: agent, parent: self(), stage_transaction: false] {:ok, pool} = P.start_link(opts) - {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], [stage_prepare: true] ++ opts) + {:ok, stage} = P.start_producer(pool, %Q{}, [:param], [stage_prepare: true] ++ opts) mon = Process.monitor(stage) assert [{stage, [cancel: :transient]}] |> GenStage.stream() |> Enum.to_list() == [%R{}, %R{}] @@ -75,7 +75,7 @@ defmodule StageTest do parent = self() opts = [agent: agent, parent: parent, stage_transaction: false] {:ok, pool} = P.start_link(opts) - {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) + {:ok, stage} = P.start_producer(pool, %Q{}, [:param], opts) ref = Process.monitor(stage) send(stage, {:"$gen_producer", {parent, ref}, {:subscribe, nil, [cancel: :transient]}}) @@ -112,7 +112,7 @@ defmodule StageTest do mapper = fn(conn, res) -> [P.execute!(conn, %Q{}, res, opts), :mapped] end opts = [stream_mapper: mapper] ++ opts - {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) + {:ok, stage} = P.start_producer(pool, %Q{}, [:param], opts) mon = Process.monitor(stage) assert [{stage, [cancel: :transient]}] |> GenStage.stream() |> Enum.to_list() == [%R{}, :mapped, %R{}, :mapped] @@ -142,7 +142,7 @@ defmodule StageTest do opts = [agent: agent, parent: parent, stage_transaction: false] Process.flag(:trap_exit, true) {:ok, pool} = P.start_link(opts) - {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) + {:ok, stage} = P.start_producer(pool, %Q{}, [:param], opts) send(stage, {:"$gen_producer", {parent, make_ref()}, {:subscribe, nil, []}}) @@ -175,7 +175,7 @@ defmodule StageTest do opts = [agent: agent, parent: parent, stage_transaction: false] Process.flag(:trap_exit, true) {:ok, pool} = P.start_link(opts) - assert {:error, {^err, _}} = P.stream_stage(pool, %Q{}, [:param], opts) + assert {:error, {^err, _}} = P.start_producer(pool, %Q{}, [:param], opts) assert_receive :reconnected @@ -205,7 +205,7 @@ defmodule StageTest do assert_receive {:hi, conn} Process.flag(:trap_exit, true) - {:error, {%DBConnection.ConnectionError{}, _} = reason} = P.stream_stage(pool, %Q{}, [:param], opts) + {:error, {%DBConnection.ConnectionError{}, _} = reason} = P.start_producer(pool, %Q{}, [:param], opts) assert_receive {:EXIT, stage, ^reason} @@ -236,7 +236,7 @@ defmodule StageTest do opts = [agent: agent, parent: parent, stage_transaction: false] {:ok, pool} = P.start_link(opts) Process.flag(:trap_exit, true) - {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) + {:ok, stage} = P.start_producer(pool, %Q{}, [:param], opts) catch_exit([{stage, [cancel: :transient]}] |> GenStage.stream() |> Enum.to_list()) assert_receive {:EXIT, ^stage, {^err, _}} @@ -269,7 +269,7 @@ defmodule StageTest do opts = [agent: agent, parent: parent, stage_transaction: false] {:ok, pool} = P.start_link(opts) Process.flag(:trap_exit, true) - {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) + {:ok, stage} = P.start_producer(pool, %Q{}, [:param], opts) catch_exit([{stage, [cancel: :transient]}] |> GenStage.stream() |> Enum.to_list()) assert_receive {:EXIT, ^stage, {^err, _}} @@ -299,7 +299,7 @@ defmodule StageTest do opts = [agent: agent, parent: parent, stage_transaction: false] {:ok, pool} = P.start_link(opts) Process.flag(:trap_exit, true) - {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) + {:ok, stage} = P.start_producer(pool, %Q{}, [:param], opts) catch_exit([{stage, [cancel: :transient]}] |> GenStage.stream() |> Enum.to_list()) assert_receive {:EXIT, ^stage, {^err, _}} @@ -333,7 +333,7 @@ defmodule StageTest do opts = [agent: agent, parent: parent, stage_transaction: false] {:ok, pool} = P.start_link(opts) Process.flag(:trap_exit, true) - {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) + {:ok, stage} = P.start_producer(pool, %Q{}, [:param], opts) catch_exit([{stage, [cancel: :transient]}] |> GenStage.stream() |> Enum.to_list()) assert_receive {:EXIT, ^stage, {^err, _}} diff --git a/integration_test/cases/transaction_stage_test.exs b/integration_test/cases/transaction_producer_test.exs similarity index 93% rename from integration_test/cases/transaction_stage_test.exs rename to integration_test/cases/transaction_producer_test.exs index bb181a28..06145014 100644 --- a/integration_test/cases/transaction_stage_test.exs +++ b/integration_test/cases/transaction_producer_test.exs @@ -1,4 +1,4 @@ -defmodule TransactionStageTest do +defmodule TransactionProducerTest do use ExUnit.Case, async: true alias TestPool, as: P @@ -21,7 +21,7 @@ defmodule TransactionStageTest do opts = [agent: agent, parent: self()] {:ok, pool} = P.start_link(opts) - {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) + {:ok, stage} = P.start_producer(pool, %Q{}, [:param], opts) mon = Process.monitor(stage) assert [{stage, [cancel: :transient]}] |> GenStage.stream() |> Enum.to_list() == [%R{}, %R{}] @@ -53,7 +53,7 @@ defmodule TransactionStageTest do opts = [agent: agent, parent: self()] {:ok, pool} = P.start_link(opts) - {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], [stage_prepare: true] ++ opts) + {:ok, stage} = P.start_producer(pool, %Q{}, [:param], [stage_prepare: true] ++ opts) mon = Process.monitor(stage) assert [{stage, [cancel: :transient]}] |> GenStage.stream() |> Enum.to_list() == [%R{}, %R{}] @@ -85,7 +85,7 @@ defmodule TransactionStageTest do parent = self() opts = [agent: agent, parent: parent] {:ok, pool} = P.start_link(opts) - {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) + {:ok, stage} = P.start_producer(pool, %Q{}, [:param], opts) ref = Process.monitor(stage) send(stage, {:"$gen_producer", {parent, ref}, {:subscribe, nil, [cancel: :transient]}}) @@ -126,7 +126,7 @@ defmodule TransactionStageTest do mapper = fn(conn, res) -> [P.execute!(conn, %Q{}, res, opts), :mapped] end opts = [stream_mapper: mapper] ++ opts - {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) + {:ok, stage} = P.start_producer(pool, %Q{}, [:param], opts) mon = Process.monitor(stage) assert [{stage, [cancel: :transient]}] |> GenStage.stream() |> Enum.to_list() == [%R{}, :mapped, %R{}, :mapped] @@ -159,7 +159,7 @@ defmodule TransactionStageTest do opts = [agent: agent, parent: parent] Process.flag(:trap_exit, true) {:ok, pool} = P.start_link(opts) - {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) + {:ok, stage} = P.start_producer(pool, %Q{}, [:param], opts) send(stage, {:"$gen_producer", {parent, make_ref()}, {:subscribe, nil, []}}) @@ -192,7 +192,7 @@ defmodule TransactionStageTest do opts = [agent: agent, parent: parent] Process.flag(:trap_exit, true) {:ok, pool} = P.start_link(opts) - assert {:error, {^err, _}} = P.stream_stage(pool, %Q{}, [:param], opts) + assert {:error, {^err, _}} = P.start_producer(pool, %Q{}, [:param], opts) assert_receive :reconnected @@ -224,7 +224,7 @@ defmodule TransactionStageTest do assert_receive {:hi, conn} Process.flag(:trap_exit, true) - {:error, {%DBConnection.ConnectionError{}, _} = reason} = P.stream_stage(pool, %Q{}, [:param], opts) + {:error, {%DBConnection.ConnectionError{}, _} = reason} = P.start_producer(pool, %Q{}, [:param], opts) assert_receive {:EXIT, stage, ^reason} @@ -257,7 +257,7 @@ defmodule TransactionStageTest do opts = [agent: agent, parent: parent] {:ok, pool} = P.start_link(opts) Process.flag(:trap_exit, true) - {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) + {:ok, stage} = P.start_producer(pool, %Q{}, [:param], opts) catch_exit([{stage, [cancel: :transient]}] |> GenStage.stream() |> Enum.to_list()) assert_receive {:EXIT, ^stage, {^err, _}} @@ -290,7 +290,7 @@ defmodule TransactionStageTest do opts = [agent: agent, parent: parent] {:ok, pool} = P.start_link(opts) Process.flag(:trap_exit, true) - {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) + {:ok, stage} = P.start_producer(pool, %Q{}, [:param], opts) catch_exit([{stage, [cancel: :transient]}] |> GenStage.stream() |> Enum.to_list()) assert_receive {:EXIT, ^stage, {^err, _}} @@ -322,7 +322,7 @@ defmodule TransactionStageTest do opts = [agent: agent, parent: parent] {:ok, pool} = P.start_link(opts) Process.flag(:trap_exit, true) - {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) + {:ok, stage} = P.start_producer(pool, %Q{}, [:param], opts) catch_exit([{stage, [cancel: :transient]}] |> GenStage.stream() |> Enum.to_list()) assert_receive {:EXIT, ^stage, {^err, _}} @@ -356,7 +356,7 @@ defmodule TransactionStageTest do opts = [agent: agent, parent: parent] {:ok, pool} = P.start_link(opts) Process.flag(:trap_exit, true) - {:ok, stage} = P.stream_stage(pool, %Q{}, [:param], opts) + {:ok, stage} = P.start_producer(pool, %Q{}, [:param], opts) catch_exit([{stage, [cancel: :transient]}] |> GenStage.stream() |> Enum.to_list()) assert_receive {:EXIT, ^stage, {^err, _}} diff --git a/integration_test/tests.exs b/integration_test/tests.exs index 4f42d19e..e0eefbc4 100644 --- a/integration_test/tests.exs +++ b/integration_test/tests.exs @@ -10,8 +10,8 @@ Code.require_file "cases/prepare_execute_test.exs", __DIR__ Code.require_file "cases/prepare_stream_test.exs", __DIR__ Code.require_file "cases/prepare_test.exs", __DIR__ Code.require_file "cases/queue_test.exs", __DIR__ -Code.require_file "cases/stage_test.exs", __DIR__ +Code.require_file "cases/producer_test.exs", __DIR__ Code.require_file "cases/stream_test.exs", __DIR__ Code.require_file "cases/transaction_execute_test.exs", __DIR__ -Code.require_file "cases/transaction_stage_test.exs", __DIR__ +Code.require_file "cases/transaction_producer_test.exs", __DIR__ Code.require_file "cases/transaction_test.exs", __DIR__ diff --git a/lib/db_connection/stage.ex b/lib/db_connection/producer.ex similarity index 83% rename from lib/db_connection/stage.ex rename to lib/db_connection/producer.ex index 9e2e6416..2d27212e 100644 --- a/lib/db_connection/stage.ex +++ b/lib/db_connection/producer.ex @@ -1,4 +1,4 @@ -defmodule DBConnection.Stage do +defmodule DBConnection.Producer do @moduledoc """ A `GenStage` producer that streams the result of a query, optionally encapsulated in a transaction. @@ -29,7 +29,7 @@ defmodule DBConnection.Stage do and `handle_rollback/2`. In addition, the demand will be passed to `handle_first/4` and `handle_next/4` by adding `fetch: demand` to the options. """ - alias __MODULE__, as: Stage + alias __MODULE__, as: Producer use GenStage @@ -52,7 +52,7 @@ defmodule DBConnection.Stage do query = %Query{statement: "SELECT id FROM table"} opts = [stream_mapper: &Map.fetch!(&1, :rows)] - {:ok, stage} = DBConnection.Stage.start_link(pool, query, [], opts) + {:ok, stage} = DBConnection.Producer.start_link(pool, query, [], opts) stage |> GenStage.stream() |> Enum.to_list() """ def start_link(pool, query, params, opts \\ []) do @@ -66,14 +66,14 @@ defmodule DBConnection.Stage do stage_opts = Keyword.take(opts, @stage_opts) stage = init(pool, opts) state = run(&declare(&1, query, params, opts), opts, stage) - {:producer, %Stage{stage | state: state}, stage_opts} + {:producer, %Producer{stage | state: state}, stage_opts} end @doc false def handle_info(:stop, stage) do {:stop, :normal, stage} end - def handle_info({:fetch, conn, pending}, %Stage{conn: conn} = stage) do + def handle_info({:fetch, conn, pending}, %Producer{conn: conn} = stage) do handle_demand(pending, stage) end def handle_info(_, stage) do @@ -82,24 +82,24 @@ defmodule DBConnection.Stage do @doc false def handle_demand(demand, stage) do - %Stage{conn: conn, state: state, opts: opts} = stage + %Producer{conn: conn, state: state, opts: opts} = stage case run(&fetch(&1, demand, state, opts), opts, stage) do {:halt, state} -> GenStage.async_info(self(), :stop) - {:noreply, [], %Stage{stage | state: state}} + {:noreply, [], %Producer{stage | state: state}} {events, state} -> # stream_mapper may not produce the desired number of events, i.e. at # the end of the results, so we can close the cursor as soon as # possible. pending = demand - length(events) _ = if pending > 0, do: send(self(), {:fetch, conn, pending}) - {:noreply, events, %Stage{stage | state: state}} + {:noreply, events, %Producer{stage | state: state}} end end @doc false - def terminate(reason, %Stage{transaction?: true} = stage) do - %Stage{conn: conn, state: state, opts: opts} = stage + def terminate(reason, %Producer{transaction?: true} = stage) do + %Producer{conn: conn, state: state, opts: opts} = stage deallocate = &deallocate(&1, reason, state, opts) case DBConnection.transaction(conn, deallocate, opts) do {:ok, :normal} -> @@ -110,8 +110,8 @@ defmodule DBConnection.Stage do :ok end end - def terminate(reason, %Stage{transaction?: false} = stage) do - %Stage{conn: conn, state: state, opts: opts} = stage + def terminate(reason, %Producer{transaction?: false} = stage) do + %Producer{conn: conn, state: state, opts: opts} = stage try do deallocate(conn, reason, state, opts) after @@ -125,14 +125,14 @@ defmodule DBConnection.Stage do case Keyword.get(opts, :stage_transaction, true) do true -> conn = DBConnection.checkout_begin(pool, opts) - %Stage{conn: conn, transaction?: true, state: :declare, opts: opts} + %Producer{conn: conn, transaction?: true, state: :declare, opts: opts} false -> conn = DBConnection.checkout(pool, opts) - %Stage{conn: conn, transaction?: false, state: :declare, opts: opts} + %Producer{conn: conn, transaction?: false, state: :declare, opts: opts} end end - defp run(fun, opts, %Stage{conn: conn, transaction?: true}) do + defp run(fun, opts, %Producer{conn: conn, transaction?: true}) do case DBConnection.transaction(conn, fun, opts) do {:ok, result} -> result @@ -140,7 +140,7 @@ defmodule DBConnection.Stage do exit(reason) end end - defp run(fun, opts, %Stage{conn: conn, transaction?: false}) do + defp run(fun, opts, %Producer{conn: conn, transaction?: false}) do try do fun.(conn) catch diff --git a/test/test_support.exs b/test/test_support.exs index 55c362d3..c3058eab 100644 --- a/test/test_support.exs +++ b/test/test_support.exs @@ -63,8 +63,8 @@ defmodule TestConnection do DBConnection.close!(pool, query, opts2 ++ unquote(opts)) end - def stream_stage(pool, query, params, opts2 \\ []) do - DBConnection.Stage.start_link(pool, query, params, opts2 ++ unquote(opts)) + def start_producer(pool, query, params, opts2 \\ []) do + DBConnection.Producer.start_link(pool, query, params, opts2 ++ unquote(opts)) end def checkout_begin(pool, opts2 \\ []) do From 845b46c234a3723bc4c1af1059a86bc9e9d85af0 Mon Sep 17 00:00:00 2001 From: James Fish Date: Mon, 29 May 2017 01:59:18 +0100 Subject: [PATCH 20/20] Introduce DBConnection.Consumer --- integration_test/cases/consumer_test.exs | 35 ++++++ integration_test/tests.exs | 1 + lib/db_connection/consumer.ex | 150 +++++++++++++++++++++++ test/test_support.exs | 4 + 4 files changed, 190 insertions(+) create mode 100644 integration_test/cases/consumer_test.exs create mode 100644 lib/db_connection/consumer.ex diff --git a/integration_test/cases/consumer_test.exs b/integration_test/cases/consumer_test.exs new file mode 100644 index 00000000..0cd178e6 --- /dev/null +++ b/integration_test/cases/consumer_test.exs @@ -0,0 +1,35 @@ +defmodule ConsumerTest do + use ExUnit.Case, async: true + + alias TestPool, as: P + alias TestAgent, as: A + alias TestQuery, as: Q + alias TestResult, as: R + + test "start_link consumes events" do + stack = [ + {:ok, :state}, + {:ok, %R{}, :new_state}, + {:ok, %R{}, :newer_state}, + ] + {:ok, agent} = A.start_link(stack) + + opts = [agent: agent, parent: self(), stage_transaction: false] + {:ok, pool} = P.start_link(opts) + {:ok, stage} = P.start_consumer(pool, &P.execute!(&1, %Q{}, &2, opts), opts) + mon = Process.monitor(stage) + + {:ok, _} = + [1, 2] + |> Flow.from_enumerable() + |> Flow.into_stages([{stage, [cancel: :transient, max_demand: 1]}]) + + assert_receive {:DOWN, ^mon, :process, ^stage, :normal} + + assert [ + connect: [_], + handle_execute: [%Q{}, [1], _, :state], + handle_execute: [%Q{}, [2], _, :new_state] + ] = A.record(agent) + end +end diff --git a/integration_test/tests.exs b/integration_test/tests.exs index e0eefbc4..619b9620 100644 --- a/integration_test/tests.exs +++ b/integration_test/tests.exs @@ -2,6 +2,7 @@ Code.require_file "cases/after_connect_test.exs", __DIR__ Code.require_file "cases/backoff_test.exs", __DIR__ Code.require_file "cases/client_test.exs", __DIR__ Code.require_file "cases/close_test.exs", __DIR__ +Code.require_file "cases/consumer_test.exs", __DIR__ Code.require_file "cases/continuation_test.exs", __DIR__ Code.require_file "cases/execute_test.exs", __DIR__ Code.require_file "cases/idle_test.exs", __DIR__ diff --git a/lib/db_connection/consumer.ex b/lib/db_connection/consumer.ex new file mode 100644 index 00000000..ea14a056 --- /dev/null +++ b/lib/db_connection/consumer.ex @@ -0,0 +1,150 @@ +defmodule DBConnection.Consumer do + @moduledoc """ + A `GenStage` consumer that runs a fun for each batch of events, optionally + encapsulated in a transaction. + + ### Options + + * `:stage_transaction` - Whether the producer should encapsulate the query + in a transaction (default: `true`) + * `:pool_timeout` - The maximum time to wait for a reply when making a + synchronous call to the pool (default: `5_000`) + * `:queue` - Whether to block waiting in an internal queue for the + connection's state (boolean, default: `true`) + * `:timeout` - The maximum time that the caller is allowed the + to hold the connection's state (ignored when using a run/transaction + connection, default: `15_000`) + * `:log` - A function to log information about a call, either + a 1-arity fun or `{module, function, args}` with `DBConnection.LogEntry.t` + prepended to `args` or `nil`. See `DBConnection.LogEntry` (default: `nil`) + + The pool and connection module may support other options. All options + are passed to `handle_begin/2`, `handle_commit/2` and `handle_rollback/2`. + """ + alias __MODULE__, as: Consumer + + use GenStage + + @enforce_keys [:conn, :fun, :transaction?, :opts, :done?, :producers] + defstruct [:conn, :fun, :transaction?, :opts, :done?, :producers] + + @start_opts [:name, :spawn_opt, :debug] + @stage_opts [:subscribe_to] + + @doc """ + Start link a `GenStage` consumer thay will run an anonymous for each batch of + events, optionally inside a transaction. + + The transaction is rolled back if the process terminates with a reason other + than `:normal`. If the transaction is rolled back the process exits with the + same reason. + + For options see "Options" in the module documentation. + + ### Example + + fun = fn(conn, rows) -> + DBConnection.execute!(conn, statement, [rows]) + end + {:ok, stage} = DBConnection.Consumer.start_link(pool, fun, opts) + enum + |> Flow.from_enumerable() + |> Flow.map(&process_row/1) + |> Flow.into_stages([{stage, cancel: :transient}] + """ + def start_link(pool, fun, opts \\ []) when is_function(fun, 2) do + start_opts = Keyword.take(opts, @start_opts) + GenStage.start_link(__MODULE__, {pool, fun, opts}, start_opts) + end + + @doc false + def init({pool, fun, opts}) do + stage_opts = Keyword.take(opts, @stage_opts) + {:consumer, init(pool, fun, opts), stage_opts} + end + + @doc false + def handle_subscribe(:producer, _, {pid, ref}, consumer) do + case consumer do + %Consumer{done?: true} = consumer -> + GenStage.cancel({pid, ref}, :normal, [:noconnect]) + {:manual, consumer} + %Consumer{done?: false, producers: producers} = consumer -> + new_producers = Map.put(producers, ref, pid) + {:automatic, %Consumer{consumer | producers: new_producers}} + end + end + + @doc false + def handle_cancel(_, {_, ref}, consumer) do + %Consumer{producers: producers} = consumer + case Map.delete(producers, ref) do + new_producers when new_producers == %{} and producers != %{} -> + GenStage.async_info(self(), :stop) + {:noreply, [], %Consumer{consumer | done?: true, producers: %{}}} + new_producers -> + {:noreply, [], %Consumer{consumer | producers: new_producers}} + end + end + + @doc false + def handle_info(:stop, state) do + {:stop, :normal, state} + end + def handle_info(_msg, state) do + {:noreply, [], state} + end + + @doc false + def handle_events(events, _, %Consumer{transaction?: true} = consumer) do + %Consumer{conn: conn, fun: fun, opts: opts} = consumer + case DBConnection.transaction(conn, &fun.(&1, events), opts) do + {:ok, _} -> + {:noreply, [], consumer} + {:error, reason} -> + exit(reason) + end + end + def handle_events(events, _, %Consumer{transaction?: false} = consumer) do + %Consumer{conn: conn, fun: fun, opts: opts} = consumer + _ = DBConnection.run(conn, &fun.(&1, events), opts) + {:noreply, [], consumer} + end + + @doc false + def terminate(reason, %Consumer{transaction?: true} = stage) do + %Consumer{conn: conn, opts: opts} = stage + case DBConnection.transaction(conn, fn(_) -> reason end, opts) do + {:ok, :normal} -> + DBConnection.commit_checkin(conn, opts) + {:ok, reason} -> + DBConnection.rollback_checkin(conn, reason, opts) + {:error, :rollback} -> + :ok + end + end + def terminate(_, %Consumer{transaction?: false} = stage) do + %Consumer{conn: conn, opts: opts} = stage + DBConnection.checkin(conn, opts) + end + + ## Helpers + + defp init(pool, fun, opts) do + case Keyword.get(opts, :stage_transaction, true) do + true -> + pool + |> DBConnection.checkout_begin(opts) + |> init(true, fun, opts) + false -> + pool + |> DBConnection.checkout(opts) + |> init(false, fun, opts) + end + end + + defp init(conn, transaction?, fun, opts) do + %Consumer{conn: conn, transaction?: transaction?, fun: fun, opts: opts, + done?: false, producers: %{}} + end +end diff --git a/test/test_support.exs b/test/test_support.exs index c3058eab..b23be059 100644 --- a/test/test_support.exs +++ b/test/test_support.exs @@ -67,6 +67,10 @@ defmodule TestConnection do DBConnection.Producer.start_link(pool, query, params, opts2 ++ unquote(opts)) end + def start_consumer(pool, fun, opts2 \\ []) do + DBConnection.Consumer.start_link(pool, fun, opts2 ++ unquote(opts)) + end + def checkout_begin(pool, opts2 \\ []) do DBConnection.checkout_begin(pool, opts2 ++ unquote(opts)) end