summaryrefslogtreecommitdiff
path: root/apps/rexi/src
diff options
context:
space:
mode:
authorAdam Kocoloski <adam@cloudant.com>2010-10-25 15:46:05 -0400
committerAdam Kocoloski <adam@cloudant.com>2010-10-25 21:45:32 -0400
commitebac05f686b56791511cb9b599dfb5a742dcfc96 (patch)
tree00a789cd058f98fa014d927f094f9e6e9f91f6f2 /apps/rexi/src
parent952a85381ff4b5b34426000b1dee73c9e74becdd (diff)
use get-deps to pull down individual cloudant projects
Diffstat (limited to 'apps/rexi/src')
-rw-r--r--apps/rexi/src/rexi.app.src7
-rw-r--r--apps/rexi/src/rexi.erl105
-rw-r--r--apps/rexi/src/rexi_app.erl25
-rw-r--r--apps/rexi/src/rexi_monitor.erl54
-rw-r--r--apps/rexi/src/rexi_server.erl100
-rw-r--r--apps/rexi/src/rexi_sup.erl29
-rw-r--r--apps/rexi/src/rexi_utils.erl53
7 files changed, 0 insertions, 373 deletions
diff --git a/apps/rexi/src/rexi.app.src b/apps/rexi/src/rexi.app.src
deleted file mode 100644
index d56a182f..00000000
--- a/apps/rexi/src/rexi.app.src
+++ /dev/null
@@ -1,7 +0,0 @@
-{application, rexi, [
- {description, "Lightweight RPC server"},
- {vsn, "1.2"},
- {registered, [rexi_sup, rexi_server]},
- {applications, [kernel, stdlib]},
- {mod, {rexi_app,[]}}
-]}.
diff --git a/apps/rexi/src/rexi.erl b/apps/rexi/src/rexi.erl
deleted file mode 100644
index 0ba77bdb..00000000
--- a/apps/rexi/src/rexi.erl
+++ /dev/null
@@ -1,105 +0,0 @@
-% Copyright 2010 Cloudant
-%
-% Licensed under the Apache License, Version 2.0 (the "License"); you may not
-% use this file except in compliance with the License. You may obtain a copy of
-% the License at
-%
-% http://www.apache.org/licenses/LICENSE-2.0
-%
-% Unless required by applicable law or agreed to in writing, software
-% distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
-% WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
-% License for the specific language governing permissions and limitations under
-% the License.
-
--module(rexi).
--export([start/0, stop/0, restart/0]).
--export([cast/2, cast/3, kill/2]).
--export([reply/1, sync_reply/1, sync_reply/2]).
--export([async_server_call/2, async_server_call/3]).
-
--define(SERVER, rexi_server).
-
-start() ->
- application:start(rexi).
-
-stop() ->
- application:stop(rexi).
-
-restart() ->
- stop(), start().
-
-%% @equiv cast(Node, self(), MFA)
--spec cast(node(), {atom(), atom(), list()}) -> reference().
-cast(Node, MFA) ->
- cast(Node, self(), MFA).
-
-%% @doc Executes apply(M, F, A) on Node.
-%% You might want to use this instead of rpc:cast/4 for two reasons. First,
-%% the Caller pid and the returned reference are inserted into the remote
-%% process' dictionary as `rexi_from', so it has a way to communicate with you.
-%% Second, the remote process is monitored. If it exits with a Reason other
-%% than normal, Caller will receive a message of the form
-%% `{Ref, {rexi_EXIT, Reason}}' where Ref is the returned reference.
--spec cast(node(), pid(), {atom(), atom(), list()}) -> reference().
-cast(Node, Caller, MFA) ->
- Ref = make_ref(),
- ok = gen_server:cast({?SERVER, Node}, {doit, {Caller,Ref}, MFA}),
- Ref.
-
-%% @doc Sends an async kill signal to the remote process associated with Ref.
-%% No rexi_EXIT message will be sent.
--spec kill(node(), reference()) -> ok.
-kill(Node, Ref) ->
- ok = gen_server:cast({?SERVER, Node}, {kill, Ref}).
-
-%% @equiv async_server_call(Server, self(), Request)
--spec async_server_call(pid() | {atom(),node()}, any()) -> reference().
-async_server_call(Server, Request) ->
- async_server_call(Server, self(), Request).
-
-%% @doc Sends a properly formatted gen_server:call Request to the Server and
-%% returns the reference which the Server will include in its reply. The
-%% function acts more like cast() than call() in that the server process
-%% is not monitored. Clients who want to know if the server is alive should
-%% monitor it themselves before calling this function.
--spec async_server_call(pid() | {atom(),node()}, pid(), any()) -> reference().
-async_server_call(Server, Caller, Request) ->
- Ref = make_ref(),
- do_send(Server, {'$gen_call', {Caller,Ref}, Request}),
- Ref.
-
-%% @doc convenience function to reply to the original rexi Caller.
--spec reply(any()) -> any().
-reply(Reply) ->
- {Caller, Ref} = get(rexi_from),
- erlang:send(Caller, {Ref,Reply}).
-
-%% @equiv sync_reply(Reply, 300000)
-sync_reply(Reply) ->
- sync_reply(Reply, 300000).
-
-%% @doc convenience function to reply to caller and wait for response. Message
-%% is of the form {OriginalRef, {self(),reference()}, Reply}, which enables the
-%% original caller to respond back.
--spec sync_reply(any(), pos_integer() | infinity) -> any().
-sync_reply(Reply, Timeout) ->
- {Caller, Ref} = get(rexi_from),
- Tag = make_ref(),
- erlang:send(Caller, {Ref, {self(),Tag}, Reply}),
- receive {Tag, Response} ->
- Response
- after Timeout ->
- timeout
- end.
-
-%% internal functions %%
-
-% send a message as quickly as possible
-do_send(Dest, Msg) ->
- case erlang:send(Dest, Msg, [noconnect]) of
- noconnect ->
- spawn(erlang, send, [Dest, Msg]);
- ok ->
- ok
- end.
diff --git a/apps/rexi/src/rexi_app.erl b/apps/rexi/src/rexi_app.erl
deleted file mode 100644
index 2dd99c23..00000000
--- a/apps/rexi/src/rexi_app.erl
+++ /dev/null
@@ -1,25 +0,0 @@
-% Copyright 2010 Cloudant
-%
-% Licensed under the Apache License, Version 2.0 (the "License"); you may not
-% use this file except in compliance with the License. You may obtain a copy of
-% the License at
-%
-% http://www.apache.org/licenses/LICENSE-2.0
-%
-% Unless required by applicable law or agreed to in writing, software
-% distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
-% WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
-% License for the specific language governing permissions and limitations under
-% the License.
-
--module(rexi_app).
--behaviour(application).
--export([start/2, stop/1]).
-
--include_lib("eunit/include/eunit.hrl").
-
-start(_Type, StartArgs) ->
- rexi_sup:start_link(StartArgs).
-
-stop(_State) ->
- ok.
diff --git a/apps/rexi/src/rexi_monitor.erl b/apps/rexi/src/rexi_monitor.erl
deleted file mode 100644
index 819b6bce..00000000
--- a/apps/rexi/src/rexi_monitor.erl
+++ /dev/null
@@ -1,54 +0,0 @@
-% Copyright 2010 Cloudant
-%
-% Licensed under the Apache License, Version 2.0 (the "License"); you may not
-% use this file except in compliance with the License. You may obtain a copy of
-% the License at
-%
-% http://www.apache.org/licenses/LICENSE-2.0
-%
-% Unless required by applicable law or agreed to in writing, software
-% distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
-% WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
-% License for the specific language governing permissions and limitations under
-% the License.
-
--module(rexi_monitor).
--export([start/1, stop/1]).
-
--include_lib("eunit/include/eunit.hrl").
-
-%% @doc spawn_links a process which monitors the supplied list of items and
-%% returns the process ID. If a monitored process exits, the caller will
-%% receive a {rexi_DOWN, MonitoringPid, DeadPid, Reason} message.
--spec start([pid() | atom() | {atom(),node()}]) -> pid().
-start(Procs) ->
- Parent = self(),
- spawn_link(fun() ->
- [erlang:monitor(process, P) || P <- Procs],
- wait_monitors(Parent)
- end).
-
-%% @doc Cleanly shut down the monitoring process and flush all rexi_DOWN
-%% messages from our mailbox.
--spec stop(pid()) -> ok.
-stop(MonitoringPid) ->
- MonitoringPid ! {self(), shutdown},
- flush_down_messages().
-
-%% internal functions %%
-
-wait_monitors(Parent) ->
- receive
- {'DOWN', _, process, Pid, Reason} ->
- Parent ! {rexi_DOWN, self(), Pid, Reason},
- wait_monitors(Parent);
- {Parent, shutdown} ->
- ok
- end.
-
-flush_down_messages() ->
- receive {rexi_DOWN, _, _, _} ->
- flush_down_messages()
- after 0 ->
- ok
- end.
diff --git a/apps/rexi/src/rexi_server.erl b/apps/rexi/src/rexi_server.erl
deleted file mode 100644
index eb9b0ca2..00000000
--- a/apps/rexi/src/rexi_server.erl
+++ /dev/null
@@ -1,100 +0,0 @@
-% Copyright 2010 Cloudant
-%
-% Licensed under the Apache License, Version 2.0 (the "License"); you may not
-% use this file except in compliance with the License. You may obtain a copy of
-% the License at
-%
-% http://www.apache.org/licenses/LICENSE-2.0
-%
-% Unless required by applicable law or agreed to in writing, software
-% distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
-% WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
-% License for the specific language governing permissions and limitations under
-% the License.
-
--module(rexi_server).
--behaviour(gen_server).
--export([init/1, handle_call/3, handle_cast/2, handle_info/2, terminate/2,
- code_change/3]).
-
--export([start_link/0, init_p/2]).
-
--include_lib("eunit/include/eunit.hrl").
-
--record(st, {
- workers = ets:new(workers, [private, {keypos,2}])
-}).
-
-start_link() ->
- gen_server:start_link({local, ?MODULE}, ?MODULE, [], []).
-
-init([]) ->
- {ok, #st{}}.
-
-handle_call(_Request, _From, St) ->
- {reply, ignored, St}.
-
-handle_cast({doit, From, MFA}, #st{workers=Workers} = St) ->
- {LocalPid, Ref} = spawn_monitor(?MODULE, init_p, [From, MFA]),
- {noreply, St#st{workers = add_worker({LocalPid, Ref, From}, Workers)}};
-
-handle_cast({kill, FromRef}, #st{workers=Workers} = St) ->
- case find_worker_from(FromRef, Workers) of
- {Pid, KeyRef, {_, FromRef}} ->
- erlang:demonitor(KeyRef),
- exit(Pid, kill),
- {noreply, St#st{workers = remove_worker(KeyRef, Workers)}};
- false ->
- {noreply, St}
- end.
-
-handle_info({'DOWN', Ref, process, _, normal}, #st{workers=Workers} = St) ->
- {noreply, St#st{workers = remove_worker(Ref, Workers)}};
-
-handle_info({'DOWN', Ref, process, Pid, Reason}, #st{workers=Workers} = St) ->
- case find_worker(Ref, Workers) of
- {Pid, Ref, From} ->
- notify_caller(From, Reason),
- {noreply, St#st{workers = remove_worker(Ref, Workers)}};
- false ->
- {noreply, St}
- end;
-
-handle_info(_Info, St) ->
- {noreply, St}.
-
-terminate(_Reason, St) ->
- ets:foldl(fun({Pid, _, _}, _) -> exit(Pid,kill) end, nil, St#st.workers),
- ok.
-
-code_change(_OldVsn, St, _Extra) ->
- {ok, St}.
-
-%% @doc initializes a process started by rexi_server.
--spec init_p({pid(), reference()}, {atom(), atom(), list()}) -> any().
-init_p(From, {M,F,A}) ->
- put(rexi_from, From),
- put(initial_call, {M,F,length(A)}),
- try apply(M, F, A) catch _:Reason -> exit(Reason) end.
-
-%% internal
-
-add_worker(Worker, Tab) ->
- ets:insert(Tab, Worker), Tab.
-
-remove_worker(Ref, Tab) ->
- ets:delete(Tab, Ref), Tab.
-
-find_worker(Ref, Tab) ->
- case ets:lookup(Tab, Ref) of [] -> false; [Worker] -> Worker end.
-
-find_worker_from(Ref, Tab) ->
- case ets:match_object(Tab, {'_', '_', {'_', Ref}}) of
- [] ->
- false;
- [Worker] ->
- Worker
- end.
-
-notify_caller({Caller, Ref}, Reason) ->
- Caller ! {Ref, {rexi_EXIT, Reason}}.
diff --git a/apps/rexi/src/rexi_sup.erl b/apps/rexi/src/rexi_sup.erl
deleted file mode 100644
index 828ee54d..00000000
--- a/apps/rexi/src/rexi_sup.erl
+++ /dev/null
@@ -1,29 +0,0 @@
-% Copyright 2010 Cloudant
-%
-% Licensed under the Apache License, Version 2.0 (the "License"); you may not
-% use this file except in compliance with the License. You may obtain a copy of
-% the License at
-%
-% http://www.apache.org/licenses/LICENSE-2.0
-%
-% Unless required by applicable law or agreed to in writing, software
-% distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
-% WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
-% License for the specific language governing permissions and limitations under
-% the License.
-
--module(rexi_sup).
--behaviour(supervisor).
--export([init/1]).
-
--export([start_link/1]).
-
--include_lib("eunit/include/eunit.hrl").
-
-start_link(Args) ->
- supervisor:start_link({local,?MODULE}, ?MODULE, Args).
-
-init([]) ->
- Mod = rexi_server,
- Spec = {Mod, {Mod,start_link,[]}, permanent, 100, worker, [Mod]},
- {ok, {{one_for_one, 3, 10}, [Spec]}}.
diff --git a/apps/rexi/src/rexi_utils.erl b/apps/rexi/src/rexi_utils.erl
deleted file mode 100644
index 3b6102da..00000000
--- a/apps/rexi/src/rexi_utils.erl
+++ /dev/null
@@ -1,53 +0,0 @@
--module(rexi_utils).
-
--export([recv/6]).
-
-%% @doc set up the receive loop with an overall timeout
--spec recv([any()], integer(), function(), any(), timeout(), timeout()) ->
- {ok, any()} | timeout | {error, any()}.
-recv(Refs, Keypos, Fun, Acc0, infinity, PerMsgTO) ->
- process_mailbox(Refs, Keypos, Fun, Acc0, nil, PerMsgTO);
-recv(Refs, Keypos, Fun, Acc0, GlobalTimeout, PerMsgTO) ->
- TimeoutRef = erlang:make_ref(),
- TRef = erlang:send_after(GlobalTimeout, self(), {timeout, TimeoutRef}),
- try
- process_mailbox(Refs, Keypos, Fun, Acc0, TimeoutRef, PerMsgTO)
- after
- erlang:cancel_timer(TRef)
- end.
-
-process_mailbox(RefList, Keypos, Fun, Acc0, TimeoutRef, PerMsgTO) ->
- case process_message(RefList, Keypos, Fun, Acc0, TimeoutRef, PerMsgTO) of
- {ok, Acc} ->
- process_mailbox(RefList, Keypos, Fun, Acc, TimeoutRef, PerMsgTO);
- {stop, Acc} ->
- {ok, Acc};
- Error ->
- Error
- end.
-
-process_message(RefList, Keypos, Fun, Acc0, TimeoutRef, PerMsgTO) ->
- receive
- {timeout, TimeoutRef} ->
- timeout;
- {Ref, Msg} ->
- case lists:keyfind(Ref, Keypos, RefList) of
- false ->
- % this was some non-matching message which we will ignore
- {ok, Acc0};
- Worker ->
- Fun(Msg, Worker, Acc0)
- end;
- {Ref, From, Msg} ->
- case lists:keyfind(Ref, Keypos, RefList) of
- false ->
- {ok, Acc0};
- Worker ->
- Fun(Msg, {Worker, From}, Acc0)
- end;
- {rexi_DOWN, _RexiMonPid, ServerPid, Reason} = Msg ->
- io:format("rexi_DOWN ~p ~p", [ServerPid, Reason]),
- Fun(Msg, nil, Acc0)
- after PerMsgTO ->
- timeout
- end.