summaryrefslogtreecommitdiff
path: root/src/couchdb
diff options
context:
space:
mode:
Diffstat (limited to 'src/couchdb')
-rw-r--r--src/couchdb/couch_btree.erl8
-rw-r--r--src/couchdb/couch_config.erl104
-rw-r--r--src/couchdb/couch_db.erl37
-rw-r--r--src/couchdb/couch_db.hrl28
-rw-r--r--src/couchdb/couch_db_updater.erl70
-rw-r--r--src/couchdb/couch_httpd.erl131
-rw-r--r--src/couchdb/couch_httpd_db.erl16
-rw-r--r--src/couchdb/couch_httpd_misc_handlers.erl30
-rw-r--r--src/couchdb/couch_query_servers.erl13
-rw-r--r--src/couchdb/couch_rep.erl4
-rw-r--r--src/couchdb/couch_server.erl32
-rw-r--r--src/couchdb/couch_server_sup.erl12
-rw-r--r--src/couchdb/couch_util.erl18
13 files changed, 312 insertions, 191 deletions
diff --git a/src/couchdb/couch_btree.erl b/src/couchdb/couch_btree.erl
index 29e82911..14bc4a1f 100644
--- a/src/couchdb/couch_btree.erl
+++ b/src/couchdb/couch_btree.erl
@@ -277,11 +277,11 @@ modify_node(Bt, RootPointerInfo, Actions, QueryOutput) ->
{NodeType, NodeList} = get_node(Bt, Pointer)
end,
NodeTuple = list_to_tuple(NodeList),
+
+ {ok, NewNodeList, QueryOutput2, Bt2} =
case NodeType of
- kp_node ->
- {ok, NewNodeList, QueryOutput2, Bt2} = modify_kpnode(Bt, NodeTuple, 1, Actions, [], QueryOutput);
- kv_node ->
- {ok, NewNodeList, QueryOutput2, Bt2} = modify_kvnode(Bt, NodeTuple, 1, Actions, [], QueryOutput)
+ kp_node -> modify_kpnode(Bt, NodeTuple, 1, Actions, [], QueryOutput);
+ kv_node -> modify_kvnode(Bt, NodeTuple, 1, Actions, [], QueryOutput)
end,
case NewNodeList of
[] -> % no nodes remain
diff --git a/src/couchdb/couch_config.erl b/src/couchdb/couch_config.erl
index f3b4f328..d989b31d 100644
--- a/src/couchdb/couch_config.erl
+++ b/src/couchdb/couch_config.erl
@@ -20,12 +20,14 @@
-include("couch_db.hrl").
-behaviour(gen_server).
--export([start_link/1, init/1, handle_call/3, handle_cast/2, handle_info/2,terminate/2, code_change/3]).
--export([all/0, get/1, get/2, get/3, delete/2, set/3, register/1, register/2,load_ini_file/1]).
+-export([start_link/1, init/1, handle_call/3, handle_cast/2, handle_info/2,
+ terminate/2, code_change/3]).
+-export([all/0, get/1, get/2, get/3, delete/2, set/3, set/4, register/1,
+ register/2, load_ini_file/1]).
-record(config,
{notify_funs=[],
- writeback_filename=""
+ write_filename=""
}).
%% Public API %%
@@ -36,29 +38,37 @@ start_link(IniFiles) ->
gen_server:start_link({local, ?MODULE}, ?MODULE, IniFiles, []).
all() ->
- gen_server:call(?MODULE, all).
+ lists:sort(ets:tab2list(?MODULE)).
+
+
get(Section) when is_binary(Section) ->
- gen_server:call(?MODULE, {fetch, binary_to_list(Section)});
+ ?MODULE:get(?b2l(Section));
get(Section) ->
- gen_server:call(?MODULE, {fetch, Section}).
-get(Section, Key) when is_binary(Section) and is_binary(Key) ->
- get(binary_to_list(Section), binary_to_list(Key), undefined);
+ Matches = ets:match(?MODULE, {{Section, '$1'}, '$2'}),
+ [{Key, Value} || [Key, Value] <- Matches].
+
+
get(Section, Key) ->
- get(Section, Key, undefined).
+ ?MODULE:get(Section, Key, undefined).
+
get(Section, Key, Default) when is_binary(Section) and is_binary(Key) ->
- gen_server:call(?MODULE, {fetch, binary_to_list(Section), binary_to_list(Key), Default});
+ ?MODULE:get(?b2l(Section), ?b2l(Key), Default);
get(Section, Key, Default) ->
- gen_server:call(?MODULE, {fetch, Section, Key, Default}).
+ case ets:lookup(?MODULE, {Section, Key}) of
+ [] -> Default;
+ [{_,Result}] -> Result
+ end.
-set(Section, Key, Value) when is_binary(Section) and is_binary(Key) ->
- gen_server:call(?MODULE, {set, [{{binary_to_list(Section), binary_to_list(Key)}, Value}]});
set(Section, Key, Value) ->
- gen_server:call(?MODULE, {set, [{{Section, Key}, Value}]}).
+ set(Section, Key, Value, true).
+
+set(Section, Key, Value, Persist) when is_binary(Section) and is_binary(Key) ->
+ set(?b2l(Section), ?b2l(Key), Value, Persist);
+set(Section, Key, Value, Persist) ->
+ gen_server:call(?MODULE, {set, [{{Section, Key}, Value}], Persist}).
-delete(Section, Key) when is_binary(Section) and is_binary(Key) ->
- gen_server:call(?MODULE, {delete, {binary_to_list(Section), binary_to_list(Key)}});
delete(Section, Key) ->
- gen_server:call(?MODULE, {delete, {Section, Key}}).
+ set(Section, Key, "").
register(Fun) ->
?MODULE:register(Fun, self()).
@@ -73,42 +83,36 @@ register(Fun, Pid) ->
init(IniFiles) ->
ets:new(?MODULE, [named_table, set, protected]),
[ok = load_ini_file(IniFile) || IniFile <- IniFiles],
- {ok, #config{writeback_filename=lists:last(IniFiles)}}.
-
-handle_call(all, _From, Config) ->
- {reply, lists:sort(ets:tab2list(?MODULE)), Config};
-
-handle_call({fetch, Section}, _From, Config) ->
- Matches = ets:match(?MODULE, {{Section, '$1'}, '$2'}),
- {reply, [{Key, Value} || [Key, Value] <- Matches], Config};
-
-handle_call({fetch, Section, Key, Default}, _From, Config) ->
- Ret = case ets:lookup(?MODULE, {Section, Key}) of
- [] -> Default;
- [{_,Result}] -> Result
- end,
- {reply, Ret, Config};
-
-handle_call({set, KVs}, _From, Config) ->
- [ok = insert_and_commit(Config, KV) || KV <- KVs],
- {reply, ok, Config};
-
-handle_call({delete, Key}, _From, Config) ->
- ets:delete(?MODULE, Key),
+ {ok, #config{write_filename=lists:last(IniFiles)}}.
+
+handle_call({set, KVs, Persist}, _From, Config) ->
+ lists:map(
+ fun({{Section, Key}, Value}=KV) ->
+ true = ets:insert(?MODULE, KV),
+ if Persist ->
+ ok = couch_config_writer:save_to_file(KV,
+ Config#config.write_filename);
+ true -> ok
+ end,
+ [catch F(Section, Key, Value)
+ || {_Pid, F} <- Config#config.notify_funs]
+ end, KVs),
{reply, ok, Config};
handle_call({register, Fun, Pid}, _From, #config{notify_funs=PidFuns}=Config) ->
erlang:monitor(process, Pid),
- {reply, ok, Config#config{notify_funs=[{Pid, Fun}|PidFuns]}}.
-
-%% @spec insert_and_commit(Tab::etstable(), Config::any()) -> ok
-%% @doc Inserts a Key/Value pair into the ets table, writes it to the storage
-%% ini file and calls all registered callback functions for Key.
-insert_and_commit(Config, KV) ->
- true = ets:insert(?MODULE, KV),
- % notify funs
- [catch Fun(KV) || {_Pid, Fun} <- Config#config.notify_funs],
- couch_config_writer:save_to_file(KV, Config#config.writeback_filename).
+ % convert 1 and 2 arity to 3 arity
+ Fun2 =
+ if is_function(Fun, 1) ->
+ fun(Section, _Key, _Value) -> Fun(Section) end;
+ is_function(Fun, 2) ->
+ fun(Section, Key, _Value) -> Fun(Section, Key) end;
+ is_function(Fun, 3) ->
+ Fun
+ end,
+ {reply, ok, Config#config{notify_funs=[{Pid, Fun2}|PidFuns]}}.
+
+
%% @spec load_ini_file(IniFile::filename()) -> ok
%% @doc Parses an ini file and stores Key/Value Pairs into the ets table.
@@ -120,7 +124,7 @@ load_ini_file(IniFile) ->
IniBin0;
{error, enoent} ->
Msg = io_lib:format("Couldn't find server configuration file ~s.", [IniFilename]),
- io:format("~s~n", [Msg]),
+ ?LOG_ERROR("~s~n", [Msg]),
throw({startup_error, Msg})
end,
diff --git a/src/couchdb/couch_db.erl b/src/couchdb/couch_db.erl
index 6441e2e1..7de5b8db 100644
--- a/src/couchdb/couch_db.erl
+++ b/src/couchdb/couch_db.erl
@@ -21,7 +21,7 @@
-export([enum_docs/4,enum_docs/5,enum_docs_since/4,enum_docs_since/5]).
-export([enum_docs_since_reduce_to_count/1,enum_docs_reduce_to_count/1]).
-export([increment_update_seq/1,get_purge_seq/1,purge_docs/2,get_last_purged/1]).
--export([start_link/3,make_doc/2]).
+-export([start_link/3,make_doc/2,set_admins/2,get_admins/1]).
-export([init/1,terminate/2,handle_call/3,handle_cast/2,code_change/3,handle_info/2]).
@@ -67,9 +67,9 @@ open(DbName, Options) ->
close(#db{fd=Fd}) ->
couch_file:drop_ref(Fd).
-open_ref_counted(MainPid, OpeningPid, UserCred) ->
+open_ref_counted(MainPid, OpeningPid, UserCtx) ->
{ok, Db} = gen_server:call(MainPid, {open_ref_counted_instance, OpeningPid}),
- {ok, Db#db{user_ctx=UserCred}}.
+ {ok, Db#db{user_ctx=UserCtx}}.
num_refs(MainPid) ->
gen_server:call(MainPid, num_refs).
@@ -172,6 +172,16 @@ get_db_info(Db) ->
],
{ok, InfoList}.
+get_admins(#db{admins=Admins}) ->
+ Admins.
+
+set_admins(#db{update_pid=UpdatePid,user_ctx=Ctx},
+ Admins) when is_list(Admins) ->
+ case gen_server:call(UpdatePid, {set_admins, Admins, Ctx}, infinity) of
+ ok -> ok;
+ Error -> throw(Error)
+ end.
+
name(#db{name=Name}) ->
Name.
@@ -203,15 +213,28 @@ group_alike_docs([Doc|Rest], [Bucket|RestBuckets]) ->
group_alike_docs(Rest, [[Doc]|[Bucket|RestBuckets]])
end.
+
+validate_doc_update(#db{user_ctx=UserCtx, admins=Admins},
+ #doc{id= <<"_design/",_/binary>>}=Doc, _GetDiskDocFun) ->
+ UserNames = [UserCtx#user_ctx.name | UserCtx#user_ctx.roles],
+ % if the user is a server admin or db admin, allow the save
+ case length(UserNames -- [<<"_admin">> | Admins]) == length(UserNames) of
+ true ->
+ % not an admin
+ throw({unauthorized, <<"You are not a server or database admin.">>});
+ false ->
+ Doc
+ end;
validate_doc_update(#db{validate_doc_funs=[]}, Doc, _GetDiskDocFun) ->
Doc;
-validate_doc_update(_Db, #doc{id= <<"_design/",_/binary>>}=Doc, _GetDiskDocFun) ->
- Doc;
validate_doc_update(_Db, #doc{id= <<"_local/",_/binary>>}=Doc, _GetDiskDocFun) ->
Doc;
-validate_doc_update(#db{name=DbName,user_ctx={CtxProps}}=Db, Doc, GetDiskDocFun) ->
+validate_doc_update(#db{name=DbName,user_ctx=Ctx}=Db, Doc, GetDiskDocFun) ->
DiskDoc = GetDiskDocFun(),
- [case Fun(Doc, DiskDoc, {[{<<"db">>, DbName} | CtxProps]}) of
+ JsonCtx = {[{<<"db">>, DbName},
+ {<<"name">>,Ctx#user_ctx.name},
+ {<<"roles">>,Ctx#user_ctx.roles}]},
+ [case Fun(Doc, DiskDoc, JsonCtx) of
ok -> ok;
Error -> throw(Error)
end || Fun <- Db#db.validate_doc_funs],
diff --git a/src/couchdb/couch_db.hrl b/src/couchdb/couch_db.hrl
index 6769c840..7bf3cb9d 100644
--- a/src/couchdb/couch_db.hrl
+++ b/src/couchdb/couch_db.hrl
@@ -85,19 +85,35 @@
meta = []
}).
-
+-record(user_ctx,
+ {name=nil,
+ roles=[]
+ }).
+
+% This should be updated anytime a header change happens that requires more
+% than filling in new defaults.
+%
+% As long the changes are limited to new header fields (with inline
+% defaults) added to the end of the file, then there is no need to increment
+% the disk revision number.
+%
+% if the disk revision is incremented, then new upgrade logic will need to be
+% added to couch_db_updater:init_db.
+
+-define(LATEST_DISK_VERSION, 0).
-record(db_header,
- {write_version = 0,
+ {disk_version = ?LATEST_DISK_VERSION,
update_seq = 0,
summary_stream_state = nil,
fulldocinfo_by_id_btree_state = nil,
docinfo_by_seq_btree_state = nil,
local_docs_btree_state = nil,
purge_seq = 0,
- purged_docs = nil
+ purged_docs = nil,
+ admins_ptr = nil
}).
-record(db,
@@ -114,9 +130,11 @@
name,
filepath,
validate_doc_funs=[],
- user_ctx={[]}
+ admins=[],
+ admins_ptr=nil,
+ user_ctx=#user_ctx{}
}).
-
+
-record(view_query_args, {
start_key = nil,
diff --git a/src/couchdb/couch_db_updater.erl b/src/couchdb/couch_db_updater.erl
index fecc5a14..67a6f624 100644
--- a/src/couchdb/couch_db_updater.erl
+++ b/src/couchdb/couch_db_updater.erl
@@ -60,6 +60,19 @@ handle_call(increment_update_seq, _From, Db) ->
couch_db_update_notifier:notify({updated, Db#db.name}),
{reply, {ok, Db2#db.update_seq}, Db2};
+handle_call({set_admins, NewAdmins, #user_ctx{roles=Roles}}, _From, Db) ->
+ DbAdmins = [<<"_admin">> | Db#db.admins],
+ case length(DbAdmins -- Roles) == length(DbAdmins) of
+ true ->
+ {reply, {unauthorized, <<"You are not a db or server admin.">>}, Db};
+ false ->
+ {ok, Ptr} = couch_file:append_term(Db#db.fd, NewAdmins),
+ Db2 = commit_data(Db#db{admins=NewAdmins, admins_ptr=Ptr,
+ update_seq=Db#db.update_seq+1}),
+ ok = gen_server:call(Db2#db.main_pid, {db_updated, Db2}),
+ {reply, ok, Db2}
+ end;
+
handle_call({purge_docs, _IdRevs}, _From,
#db{compactor_pid=Pid}=Db) when Pid /= nil ->
{reply, {error, purge_during_compaction}, Db};
@@ -128,7 +141,7 @@ handle_cast(start_compact, Db) ->
case Db#db.compactor_pid of
nil ->
?LOG_INFO("Starting compaction for db \"~s\"", [Db#db.name]),
- Pid = spawn_link(fun() -> start_copy_compact_int(Db) end),
+ Pid = spawn_link(fun() -> start_copy_compact(Db) end),
Db2 = Db#db{compactor_pid=Pid},
ok = gen_server:call(Db#db.main_pid, {db_updated, Db2}),
{noreply, Db2};
@@ -166,7 +179,7 @@ handle_cast({compact_done, CompactFilepath}, #db{filepath=Filepath}=Db) ->
?LOG_INFO("Compaction file still behind main file "
"(update seq=~p. compact update seq=~p). Retrying.",
[Db#db.update_seq, NewSeq]),
- Pid = spawn_link(fun() -> start_copy_compact_int(Db) end),
+ Pid = spawn_link(fun() -> start_copy_compact(Db) end),
Db2 = Db#db{compactor_pid=Pid},
couch_file:close(NewFd),
{noreply, Db2}
@@ -222,7 +235,19 @@ btree_by_seq_reduce(reduce, DocInfos) ->
btree_by_seq_reduce(rereduce, Reds) ->
lists:sum(Reds).
-init_db(DbName, Filepath, Fd, Header) ->
+simple_upgrade_record(Old, New) when size(Old) == size(New)->
+ Old;
+simple_upgrade_record(Old, New) ->
+ NewValuesTail =
+ lists:sublist(tuple_to_list(New), size(Old) + 1, size(New)-size(Old)),
+ list_to_tuple(tuple_to_list(Old) ++ NewValuesTail).
+
+init_db(DbName, Filepath, Fd, Header0) ->
+ case element(2, Header0) of
+ ?LATEST_DISK_VERSION -> ok;
+ _ -> throw({database_disk_version_error, "Incorrect disk header version"})
+ end,
+ Header = simple_upgrade_record(Header0, #db_header{}),
{ok, SummaryStream} = couch_stream:open(Header#db_header.summary_stream_state, Fd),
ok = couch_stream:set_min_buffer(SummaryStream, 10000),
Less =
@@ -242,7 +267,13 @@ init_db(DbName, Filepath, Fd, Header) ->
{join, fun(X,Y) -> btree_by_seq_join(X,Y) end},
{reduce, fun(X,Y) -> btree_by_seq_reduce(X,Y) end}]),
{ok, LocalDocsBtree} = couch_btree:open(Header#db_header.local_docs_btree_state, Fd),
-
+ case Header#db_header.admins_ptr of
+ nil ->
+ Admins = [],
+ AdminsPtr = nil;
+ AdminsPtr ->
+ {ok, Admins} = couch_file:pread_term(Fd, AdminsPtr)
+ end,
#db{
update_pid=self(),
fd=Fd,
@@ -253,7 +284,9 @@ init_db(DbName, Filepath, Fd, Header) ->
local_docs_btree = LocalDocsBtree,
update_seq = Header#db_header.update_seq,
name = DbName,
- filepath=Filepath}.
+ filepath = Filepath,
+ admins = Admins,
+ admins_ptr = AdminsPtr}.
close_db(#db{fd=Fd,summary_stream=Ss}) ->
@@ -474,7 +507,8 @@ commit_data(#db{fd=Fd, header=Header} = Db) ->
summary_stream_state = couch_stream:get_state(Db#db.summary_stream),
docinfo_by_seq_btree_state = couch_btree:get_state(Db#db.docinfo_by_seq_btree),
fulldocinfo_by_id_btree_state = couch_btree:get_state(Db#db.fulldocinfo_by_id_btree),
- local_docs_btree_state = couch_btree:get_state(Db#db.local_docs_btree)
+ local_docs_btree_state = couch_btree:get_state(Db#db.local_docs_btree),
+ admins_ptr = Db#db.admins_ptr
},
if Header == Header2 ->
Db; % unchanged. nothing to do
@@ -532,7 +566,7 @@ copy_docs(#db{fd=SrcFd}=Db, #db{fd=DestFd,summary_stream=DestStream}=NewDb, Info
-copy_compact_docs(Db, NewDb, Retry) ->
+copy_compact(Db, NewDb, Retry) ->
EnumBySeqFun =
fun(#doc_info{update_seq=Seq}=DocInfo, _Offset, {AccNewDb, AccUncopied}) ->
case couch_util:should_flush() of
@@ -546,15 +580,19 @@ copy_compact_docs(Db, NewDb, Retry) ->
{ok, {NewDb2, Uncopied}} =
couch_btree:foldl(Db#db.docinfo_by_seq_btree, NewDb#db.update_seq + 1, EnumBySeqFun, {NewDb, []}),
- case Uncopied of
- [#doc_info{update_seq=LastSeq} | _] ->
- commit_data( copy_docs(Db, NewDb2#db{update_seq=LastSeq},
- lists:reverse(Uncopied), Retry));
- [] ->
- NewDb2
- end.
+ NewDb3 = copy_docs(Db, NewDb2, lists:reverse(Uncopied), Retry),
+
+ % copy misc header values
+ if NewDb3#db.admins /= Db#db.admins ->
+ {ok, Ptr} = couch_file:append_term(NewDb3#db.fd, Db#db.admins),
+ NewDb4 = NewDb3#db{admins=Db#db.admins, admins_ptr=Ptr};
+ true ->
+ NewDb4 = NewDb3
+ end,
+
+ commit_data(NewDb4#db{update_seq=Db#db.update_seq}).
-start_copy_compact_int(#db{name=Name,filepath=Filepath}=Db) ->
+start_copy_compact(#db{name=Name,filepath=Filepath}=Db) ->
CompactFile = Filepath ++ ".compact",
?LOG_DEBUG("Compaction process spawned for db \"~s\"", [Name]),
case couch_file:open(CompactFile) of
@@ -568,7 +606,7 @@ start_copy_compact_int(#db{name=Name,filepath=Filepath}=Db) ->
ok = couch_file:write_header(Fd, ?HEADER_SIG, Header=#db_header{})
end,
NewDb = init_db(Name, CompactFile, Fd, Header),
- NewDb2 = copy_compact_docs(Db, NewDb, Retry),
+ NewDb2 = copy_compact(Db, NewDb, Retry),
close_db(NewDb2),
gen_server:cast(Db#db.update_pid, {compact_done, CompactFile}).
diff --git a/src/couchdb/couch_httpd.erl b/src/couchdb/couch_httpd.erl
index 6c8873dd..e1c99651 100644
--- a/src/couchdb/couch_httpd.erl
+++ b/src/couchdb/couch_httpd.erl
@@ -13,17 +13,17 @@
-module(couch_httpd).
-include("couch_db.hrl").
--export([start_link/0, stop/0, handle_request/4]).
+-export([start_link/0, stop/0, handle_request/3]).
-export([header_value/2,header_value/3,qs_value/2,qs_value/3,qs/1,path/1]).
--export([check_is_admin/1,unquote/1]).
+-export([verify_is_server_admin/1,unquote/1]).
-export([parse_form/1,json_body/1,body/1,doc_etag/1]).
-export([primary_header_value/2,partition/1,serve_file/3]).
-export([start_chunked_response/3,send_chunk/2]).
-export([start_json_response/2, start_json_response/3, end_json_response/1]).
-export([send_response/4,send_method_not_allowed/2,send_error/4]).
-export([send_json/2,send_json/3,send_json/4]).
--export([default_authenticate/1]).
+-export([default_authentication_handler/1,special_test_authentication_handler/1]).
% Maximum size of document PUT request body (4GB)
@@ -37,24 +37,21 @@ start_link() ->
BindAddress = couch_config:get("httpd", "bind_address", any),
Port = couch_config:get("httpd", "port", "5984"),
- AuthenticationFun = make_arity_1_fun(
- couch_config:get("httpd", "authentication",
- "{couch_httpd, default_authenticate}")),
-
+
UrlHandlersList = lists:map(
fun({UrlKey, SpecStr}) ->
- {list_to_binary(UrlKey), make_arity_1_fun(SpecStr)}
+ {?l2b(UrlKey), make_arity_1_fun(SpecStr)}
end, couch_config:get("httpd_global_handlers")),
DbUrlHandlersList = lists:map(
fun({UrlKey, SpecStr}) ->
- {list_to_binary(UrlKey), make_arity_2_fun(SpecStr)}
+ {?l2b(UrlKey), make_arity_2_fun(SpecStr)}
end, couch_config:get("httpd_db_handlers")),
UrlHandlers = dict:from_list(UrlHandlersList),
DbUrlHandlers = dict:from_list(DbUrlHandlersList),
Loop = fun(Req)->
apply(?MODULE, handle_request,
- [Req, UrlHandlers, DbUrlHandlers, AuthenticationFun])
+ [Req, UrlHandlers, DbUrlHandlers])
end,
% and off we go
@@ -69,8 +66,6 @@ start_link() ->
?MODULE:stop();
("httpd", "port") ->
?MODULE:stop();
- ("httpd", "authentication") ->
- ?MODULE:stop();
("httpd_global_handlers", _) ->
?MODULE:stop();
("httpd_db_handlers", _) ->
@@ -79,7 +74,8 @@ start_link() ->
{ok, Pid}.
-% SpecStr is a string like "{my_module, my_fun}" or "{my_module, my_fun, foo}"
+% SpecStr is a string like "{my_module, my_fun}"
+% or "{my_module, my_fun, <<"my_arg">>}"
make_arity_1_fun(SpecStr) ->
case couch_util:parse_term(SpecStr) of
{ok, {Mod, Fun, SpecArg}} ->
@@ -101,8 +97,9 @@ stop() ->
mochiweb_http:stop(?MODULE).
-handle_request(MochiReq, UrlHandlers, DbUrlHandlers, AuthenticationFun) ->
-
+handle_request(MochiReq, UrlHandlers, DbUrlHandlers) ->
+ AuthenticationFun = make_arity_1_fun(
+ couch_config:get("httpd", "authentication_handler")),
% for the path, use the raw path with the query string and fragment
% removed, but URL quoting left intact
RawUri = MochiReq:get(raw_path),
@@ -132,7 +129,7 @@ handle_request(MochiReq, UrlHandlers, DbUrlHandlers, AuthenticationFun) ->
% Non standard HTTP verbs aren't atoms (COPY, MOVE etc) so convert when
% possible (if any module references the atom, then it's existing).
- Meth -> try list_to_existing_atom(Meth) catch _ -> Meth end
+ Meth -> couch_util:to_existing_atom(Meth)
end,
HttpReq = #httpd{
mochi_req = MochiReq,
@@ -143,14 +140,10 @@ handle_request(MochiReq, UrlHandlers, DbUrlHandlers, AuthenticationFun) ->
},
DefaultFun = fun couch_httpd_db:handle_request/1,
HandlerFun = couch_util:dict_find(HandlerKey, UrlHandlers, DefaultFun),
- CouchHeaders = [{?l2b(K), ?l2b(V)}
- || {"X-Couch-" ++ _= K,V}
- <- mochiweb_headers:to_list(MochiReq:get(headers))],
{ok, Resp} =
try
- {UserCtxProps} = AuthenticationFun(HttpReq),
- HandlerFun(HttpReq#httpd{user_ctx={UserCtxProps ++ CouchHeaders}})
+ HandlerFun(HttpReq#httpd{user_ctx=AuthenticationFun(HttpReq)})
catch
Error ->
send_error(HttpReq, Error)
@@ -165,15 +158,44 @@ handle_request(MochiReq, UrlHandlers, DbUrlHandlers, AuthenticationFun) ->
{ok, Resp}.
+special_test_authentication_handler(Req) ->
+ case header_value(Req, "WWW-Authenticate") of
+ "X-Couch-Test-Auth " ++ NamePass ->
+ % NamePass is a colon separated string: "joe schmoe:a password".
+ {ok, [Name, Pass]} = regexp:split(NamePass, ":"),
+ case {Name, Pass} of
+ {"Jan Lehnardt", "apple"} -> ok;
+ {"Christopher Lenz", "dog food"} -> ok;
+ {"Noah Slater", "biggiesmalls endian"} -> ok;
+ {"Chris Anderson", "mp3"} -> ok;
+ {"Damien Katz", "pecan pie"} -> ok;
+ {_, _} ->
+ throw({unauthorized, <<"Name or password is incorrect.">>})
+ end,
+ #user_ctx{name=?l2b(Name)};
+ _ ->
+ % No X-Couch-Test-Auth credentials sent, give admin access so the
+ % previous authentication can be restored after the test
+ #user_ctx{roles=[<<"_admin">>]}
+ end.
-default_authenticate(Req) ->
- % by default, we just assume the users credentials for basic authentication
- % are correct.
+default_authentication_handler(Req) ->
case basic_username_pw(Req) of
- {Username, _Pw} ->
- {[{<<"name">>, ?l2b(Username)}]};
+ {User, Pass} ->
+ case couch_server:is_admin(User, Pass) of
+ true ->
+ #user_ctx{name=User, roles=[<<"_admin">>]};
+ false ->
+ throw({unauthorized, <<"Name or password is incorrect.">>})
+ end;
nil ->
- {[]}
+ case couch_server:has_admins() of
+ true ->
+ #user_ctx{};
+ false ->
+ % if no admins, then everyone is admin! Yay, admin party!
+ #user_ctx{roles=[<<"_admin">>]}
+ end
end.
@@ -224,6 +246,14 @@ json_body(#httpd{mochi_req=MochiReq}) ->
doc_etag(#doc{revs=[DiskRev|_]}) ->
"\"" ++ binary_to_list(DiskRev) ++ "\"".
+verify_is_server_admin(#httpd{user_ctx=#user_ctx{roles=Roles}}) ->
+ case lists:member(<<"_admin">>, Roles) of
+ true -> ok;
+ false -> throw({unauthorized, <<"You are not a server admin.">>})
+ end.
+
+
+
basic_username_pw(Req) ->
case header_value(Req, "Authorization") of
"Basic " ++ Base64Value ->
@@ -239,27 +269,6 @@ basic_username_pw(Req) ->
nil
end.
-check_is_admin(Req) ->
- IsNamedAdmin =
- case basic_username_pw(Req) of
- {User, Pass} ->
- couch_server:is_admin(User, Pass);
- nil ->
- false
- end,
-
- case IsNamedAdmin of
- true ->
- ok;
- false ->
- case couch_server:has_admins() of
- true ->
- throw(admin_authorization_error);
- false ->
- % if no admins, then everyone is admin! Yay, admin party!
- ok
- end
- end.
start_chunked_response(#httpd{mochi_req=MochiReq}, Code, Headers) ->
{ok, MochiReq:respond({Code, Headers ++ server_header(), chunked})}.
@@ -315,11 +324,23 @@ send_error(Req, {not_found, Reason}) ->
send_error(Req, 404, <<"not_found">>, Reason);
send_error(Req, conflict) ->
send_error(Req, 412, <<"conflict">>, <<"Document update conflict.">>);
-send_error(Req, admin_authorization_error) ->
- send_json(Req, 401,
- [{"WWW-Authenticate", "Basic realm=\"administrator\""}],
- {[{<<"error">>, <<"authorization">>},
- {<<"reason">>, <<"Admin user name and password required">>}]});
+send_error(Req, {forbidden, Msg}) ->
+ send_json(Req, 403,
+ {[{<<"error">>, <<"forbidden">>},
+ {<<"reason">>, Msg}]});
+send_error(Req, {unauthorized, Msg}) ->
+ case couch_config:get("httpd", "WWW-Authenticate", nil) of
+ nil ->
+ Headers = [];
+ Type ->
+ Headers = [{"WWW-Authenticate", Type}]
+ end,
+ send_json(Req, 401, Headers,
+ {[{<<"error">>, <<"unauthorized">>},
+ {<<"reason">>, Msg}]});
+send_error(Req, {http_error, Code, Headers, Error, Reason}) ->
+ send_json(Req, Code, Headers,
+ {[{<<"error">>, Error}, {<<"reason">>, Reason}]});
send_error(Req, {user_error, {Props}}) ->
{Headers} = proplists:get_value(<<"headers">>, Props, {[]}),
send_json(Req,
@@ -351,9 +372,9 @@ send_error(Req, Code, Error, Msg) when not is_binary(Error) ->
send_error(Req, Code, Error, Msg) when not is_binary(Msg) ->
send_error(Req, Code, Error, list_to_binary(io_lib:format("~p", [Msg])));
send_error(Req, Code, Error, <<>>) ->
- send_json(Req, Code, {[{error, Error}]});
+ send_json(Req, Code, {[{<<"error">>, Error}]});
send_error(Req, Code, Error, Msg) ->
- send_json(Req, Code, {[{error, Error}, {reason, Msg}]}).
+ send_json(Req, Code, {[{<<"error">>, Error}, {<<"reason">>, Msg}]}).
diff --git a/src/couchdb/couch_httpd_db.erl b/src/couchdb/couch_httpd_db.erl
index a239ceb5..0bf9d62e 100644
--- a/src/couchdb/couch_httpd_db.erl
+++ b/src/couchdb/couch_httpd_db.erl
@@ -42,7 +42,7 @@ handle_request(#httpd{path_parts=[DbName|RestParts],method=Method,
end.
create_db_req(#httpd{user_ctx=UserCtx}=Req, DbName) ->
- ok = couch_httpd:check_is_admin(Req),
+ ok = couch_httpd:verify_is_server_admin(Req),
case couch_server:create(DbName, [{user_ctx, UserCtx}]) of
{ok, Db} ->
couch_db:close(Db),
@@ -52,7 +52,7 @@ create_db_req(#httpd{user_ctx=UserCtx}=Req, DbName) ->
end.
delete_db_req(#httpd{user_ctx=UserCtx}=Req, DbName) ->
- ok = couch_httpd:check_is_admin(Req),
+ ok = couch_httpd:verify_is_server_admin(Req),
case couch_server:delete(DbName, [{user_ctx, UserCtx}]) of
ok ->
send_json(Req, 200, {[{ok, true}]});
@@ -230,6 +230,18 @@ db_req(#httpd{method='POST',path_parts=[_,<<"_missing_revs">>]}=Req, Db) ->
db_req(#httpd{path_parts=[_,<<"_missing_revs">>]}=Req, _Db) ->
send_method_not_allowed(Req, "POST");
+db_req(#httpd{method='PUT',path_parts=[_,<<"_admins">>]}=Req,
+ Db) ->
+ Admins = couch_httpd:json_body(Req),
+ ok = couch_db:set_admins(Db, Admins),
+ send_json(Req, {[{<<"ok">>, true}]});
+
+db_req(#httpd{method='GET',path_parts=[_,<<"_admins">>]}=Req, Db) ->
+ send_json(Req, couch_db:get_admins(Db));
+
+db_req(#httpd{path_parts=[_,<<"_admins">>]}=Req, _Db) ->
+ send_method_not_allowed(Req, "PUT,GET");
+
db_req(#httpd{method='POST',path_parts=[DbName,<<"_design">>,Name|Rest]}=Req,
Db) ->
% Special case to enable using an unencoded in the URL of design docs, as
diff --git a/src/couchdb/couch_httpd_misc_handlers.erl b/src/couchdb/couch_httpd_misc_handlers.erl
index b62a4b85..766263ee 100644
--- a/src/couchdb/couch_httpd_misc_handlers.erl
+++ b/src/couchdb/couch_httpd_misc_handlers.erl
@@ -31,7 +31,8 @@
handle_welcome_req(#httpd{method='GET'}=Req, WelcomeMessage) ->
send_json(Req, {[
{couchdb, WelcomeMessage},
- {version, list_to_binary(couch_server:get_version())}
+ {version, list_to_binary(couch_server:get_version())},
+ {start_time, list_to_binary(couch_server:get_start_time())}
]});
handle_welcome_req(Req, _) ->
send_method_not_allowed(Req, "GET,HEAD").
@@ -90,10 +91,9 @@ handle_replicate_req(Req) ->
handle_restart_req(#httpd{method='POST'}=Req) ->
- ok = couch_httpd:check_is_admin(Req),
- Response = send_json(Req, {[{ok, true}]}),
- spawn(fun() -> couch_server:remote_restart() end),
- Response;
+ ok = couch_httpd:verify_is_server_admin(Req),
+ couch_server_sup:restart_core_server(),
+ send_json(Req, 200, {[{ok, true}]});
handle_restart_req(Req) ->
send_method_not_allowed(Req, "POST").
@@ -114,7 +114,7 @@ handle_uuids_req(Req) ->
% GET /_config/
% GET /_config
handle_config_req(#httpd{method='GET', path_parts=[_]}=Req) ->
- ok = couch_httpd:check_is_admin(Req),
+ ok = couch_httpd:verify_is_server_admin(Req),
Grouped = lists:foldl(fun({{Section, Key}, Value}, Acc) ->
case dict:is_key(Section, Acc) of
true ->
@@ -129,22 +129,22 @@ handle_config_req(#httpd{method='GET', path_parts=[_]}=Req) ->
send_json(Req, 200, {KVs});
% GET /_config/Section
handle_config_req(#httpd{method='GET', path_parts=[_,Section]}=Req) ->
- ok = couch_httpd:check_is_admin(Req),
+ ok = couch_httpd:verify_is_server_admin(Req),
KVs = [{list_to_binary(Key), list_to_binary(Value)}
|| {Key, Value} <- couch_config:get(Section)],
send_json(Req, 200, {KVs});
% PUT /_config/Section/Key
% "value"
handle_config_req(#httpd{method='PUT', path_parts=[_, Section, Key]}=Req) ->
- ok = couch_httpd:check_is_admin(Req),
- Value = binary_to_list(couch_httpd:body(Req)),
- ok = couch_config:set(Section, Key, Value),
- send_json(Req, 200, {[
- {ok, true}
- ]});
+ ok = couch_httpd:verify_is_server_admin(Req),
+ Value = couch_httpd:json_body(Req),
+ Persist = couch_httpd:header_value(Req, "X-Couch-Persist") /= "false",
+ OldValue = couch_config:get(Section, Key, null),
+ ok = couch_config:set(Section, Key, ?b2l(Value), Persist),
+ send_json(Req, 200, list_to_binary(OldValue));
% GET /_config/Section/Key
handle_config_req(#httpd{method='GET', path_parts=[_, Section, Key]}=Req) ->
- ok = couch_httpd:check_is_admin(Req),
+ ok = couch_httpd:verify_is_server_admin(Req),
case couch_config:get(Section, Key, null) of
null ->
throw({not_found, unknown_config_value});
@@ -153,7 +153,7 @@ handle_config_req(#httpd{method='GET', path_parts=[_, Section, Key]}=Req) ->
end;
% DELETE /_config/Section/Key
handle_config_req(#httpd{method='DELETE',path_parts=[_,Section,Key]}=Req) ->
- ok = couch_httpd:check_is_admin(Req),
+ ok = couch_httpd:verify_is_server_admin(Req),
case couch_config:get(Section, Key, null) of
null ->
throw({not_found, unknown_config_value});
diff --git a/src/couchdb/couch_query_servers.erl b/src/couchdb/couch_query_servers.erl
index 141c9406..8465a632 100644
--- a/src/couchdb/couch_query_servers.erl
+++ b/src/couchdb/couch_query_servers.erl
@@ -69,9 +69,9 @@ prompt(Port, Json) ->
true = port_command(Port, Bin),
case read_json(Port) of
{[{<<"error">>, Id}, {<<"reason">>, Reason}]} ->
- throw({list_to_atom(binary_to_list(Id)),Reason});
+ throw({Id,Reason});
{[{<<"reason">>, Reason}, {<<"error">>, Id}]} ->
- throw({list_to_atom(binary_to_list(Id)),Reason});
+ throw({Id,Reason});
Result ->
Result
end.
@@ -181,15 +181,16 @@ validate_doc_update(Lang, FunSrc, EditDoc, DiskDoc, Ctx) ->
if DiskDoc == nil ->
null;
true ->
- couch_doc:to_json_obj(EditDoc, [revs])
+ couch_doc:to_json_obj(DiskDoc, [revs])
end,
try prompt(Port,
[<<"validate">>, FunSrc, JsonEditDoc, JsonDiskDoc, Ctx]) of
1 ->
ok;
- {ErrorObject} ->
- {user_error,
- {ErrorObject}}
+ {[{<<"forbidden">>, Message}]} ->
+ throw({forbidden, Message});
+ {[{<<"unauthorized">>, Message}]} ->
+ throw({unauthorized, Message})
after
return_linked_port(Lang, Port)
end.
diff --git a/src/couchdb/couch_rep.erl b/src/couchdb/couch_rep.erl
index eda62c86..08d4df8f 100644
--- a/src/couchdb/couch_rep.erl
+++ b/src/couchdb/couch_rep.erl
@@ -340,8 +340,8 @@ update_docs(Db, Docs, Options, NewEdits) ->
open_doc(#http_db{uri=DbUrl, headers=Headers}, DocId, Options) ->
[] = Options,
case do_http_request(DbUrl ++ url_encode(DocId), get, Headers) of
- {[{<<"error">>, ErrId}, {<<"reason">>, Reason}]} -> % binaries?
- {list_to_atom(binary_to_list(ErrId)), Reason};
+ {[{<<"error">>, ErrId}, {<<"reason">>, Reason}]} ->
+ {couch_util:to_existing_atom(ErrId), Reason};
Doc ->
{ok, couch_doc:from_json_obj(Doc)}
end;
diff --git a/src/couchdb/couch_server.erl b/src/couchdb/couch_server.erl
index a81ca5e1..34aa16b7 100644
--- a/src/couchdb/couch_server.erl
+++ b/src/couchdb/couch_server.erl
@@ -18,7 +18,7 @@
-export([open/2,create/2,delete/2,all_databases/0,get_version/0]).
-export([init/1, handle_call/3,sup_start_link/0]).
-export([handle_cast/2,code_change/3,handle_info/2,terminate/2]).
--export([dev_start/0,remote_restart/0,is_admin/2,has_admins/0]).
+-export([dev_start/0,is_admin/2,has_admins/0,get_start_time/0]).
-include("couch_db.hrl").
@@ -26,7 +26,8 @@
root_dir = [],
dbname_regexp,
max_dbs_open=100,
- current_dbs_open=0
+ current_dbs_open=0,
+ start_time=""
}).
start() ->
@@ -62,6 +63,10 @@ get_version() ->
"0.0.0"
end.
+get_start_time() ->
+ {ok, #server{start_time=Time}} = gen_server:call(couch_server, get_server),
+ Time.
+
sup_start_link() ->
gen_server:start_link({local, couch_server}, couch_server, [], []).
@@ -74,9 +79,6 @@ create(DbName, Options) ->
delete(DbName, Options) ->
gen_server:call(couch_server, {delete, DbName, Options}).
-remote_restart() ->
- gen_server:call(couch_server, remote_restart).
-
check_dbname(#server{dbname_regexp=RegExp}, DbName) ->
case regexp:match(DbName, RegExp) of
nomatch ->
@@ -137,13 +139,14 @@ init([]) ->
process_flag(trap_exit, true),
{ok, #server{root_dir=RootDir,
dbname_regexp=RegExp,
- max_dbs_open=MaxDbsOpen}}.
+ max_dbs_open=MaxDbsOpen,
+ start_time=httpd_util:rfc1123_date()}}.
terminate(_Reason, _Server) ->
ok.
all_databases() ->
- {ok, Root} = gen_server:call(couch_server, get_root),
+ {ok, #server{root_dir=Root}} = gen_server:call(couch_server, get_server),
Filenames =
filelib:fold_files(Root, "^[a-z0-9\\_\\$()\\+\\-]*[\\.]couch$", true,
fun(Filename, AccIn) ->
@@ -196,10 +199,7 @@ try_close_lru(StartTime) ->
end.
handle_call(get_server, _From, Server) ->
- {reply, Server, Server};
-
-handle_call(get_root, _From, #server{root_dir=Root}=Server) ->
- {reply, {ok, Root}, Server};
+ {reply, {ok, Server}, Server};
handle_call({open, DbName, Options}, {FromPid,_}, Server) ->
DbNameList = binary_to_list(DbName),
UserCtx = proplists:get_value(user_ctx, Options, nil),
@@ -295,15 +295,7 @@ handle_call({delete, DbName, Options}, _From, Server) ->
end;
Error ->
{reply, Error, Server}
- end;
-handle_call(remote_restart, _From, Server) ->
- case couch_config:get("couchdb", "allow_remote_restart", "false") of
- "true" ->
- exit(couch_server_sup, restart);
- _ ->
- ok
- end,
- {reply, ok, Server}.
+ end.
handle_cast(Msg, _Server) ->
exit({unknown_cast_message, Msg}).
diff --git a/src/couchdb/couch_server_sup.erl b/src/couchdb/couch_server_sup.erl
index b75747d7..9c74a1a3 100644
--- a/src/couchdb/couch_server_sup.erl
+++ b/src/couchdb/couch_server_sup.erl
@@ -14,7 +14,9 @@
-behaviour(supervisor).
--export([start_link/1,stop/0,couch_config_start_link_wrapper/2,start_primary_services/0,start_secondary_services/0]).
+-export([start_link/1,stop/0, couch_config_start_link_wrapper/2,
+ start_primary_services/0,start_secondary_services/0,
+ restart_core_server/0]).
-include("couch_db.hrl").
@@ -29,6 +31,10 @@ start_link(IniFiles) ->
{error, already_started}
end.
+restart_core_server() ->
+ supervisor:terminate_child(couch_primary_services, couch_server),
+ supervisor:restart_child(couch_primary_services, couch_server).
+
couch_config_start_link_wrapper(IniFiles, FirstConfigPid) ->
case is_process_alive(FirstConfigPid) of
true ->
@@ -119,7 +125,7 @@ start_server(IniFiles) ->
{ok, Pid}.
start_primary_services() ->
- supervisor:start_link(couch_server_sup,
+ supervisor:start_link({local, couch_primary_services}, couch_server_sup,
{{one_for_one, 10, 3600},
[{couch_log,
{couch_log, start_link, []},
@@ -156,7 +162,7 @@ start_secondary_services() ->
|| {Name, SpecStr}
<- couch_config:get("daemons"), SpecStr /= ""],
- supervisor:start_link(couch_server_sup,
+ supervisor:start_link({local, couch_secondary_services}, couch_server_sup,
{{one_for_one, 10, 3600}, DaemonChildSpecs}).
stop() ->
diff --git a/src/couchdb/couch_util.erl b/src/couchdb/couch_util.erl
index e6d6226b..d0f0c0b6 100644
--- a/src/couchdb/couch_util.erl
+++ b/src/couchdb/couch_util.erl
@@ -13,7 +13,7 @@
-module(couch_util).
-export([start_driver/1]).
--export([should_flush/0, should_flush/1]).
+-export([should_flush/0, should_flush/1, to_existing_atom/1]).
-export([new_uuid/0, rand32/0, implode/2, collate/2, collate/3]).
-export([abs_pathname/1,abs_pathname/2, trim/1, ascii_lower/1]).
-export([encodeBase64/1, decodeBase64/1, to_hex/1,parse_term/1,dict_find/3]).
@@ -33,6 +33,16 @@ start_driver(LibDir) ->
exit(erl_ddll:format_error(Error))
end.
+% works like list_to_existing_atom, except can be list or binary and it
+% gives you the original value instead of an error if no existing atom.
+to_existing_atom(V) when is_list(V)->
+ try list_to_existing_atom(V) catch _ -> V end;
+to_existing_atom(V) when is_binary(V)->
+ try list_to_existing_atom(?b2l(V)) catch _ -> V end;
+to_existing_atom(V) when is_atom(V)->
+ V.
+
+
new_uuid() ->
list_to_binary(to_hex(crypto:rand_bytes(16))).
@@ -168,11 +178,7 @@ collate(A, B, Options) when is_binary(A), is_binary(B) ->
[Result] = erlang:port_control(drv_port(), Operation, Bin),
% Result is 0 for lt, 1 for eq and 2 for gt. Subtract 1 to return the
% expected typical -1, 0, 1
- Result - 1;
-
-collate(A, B, _Options) ->
- io:format("-----A,B:~p,~p~n", [A,B]),
- throw({error, badtypes}).
+ Result - 1.
should_flush() ->
should_flush(?FLUSH_MAX_MEM).