diff options
Diffstat (limited to 'src/couchdb')
49 files changed, 4726 insertions, 2068 deletions
diff --git a/src/couchdb/Makefile.am b/src/couchdb/Makefile.am index 9e909964..308a3837 100644 --- a/src/couchdb/Makefile.am +++ b/src/couchdb/Makefile.am @@ -17,7 +17,7 @@ couchlibdir = $(localerlanglibdir)/couch-$(version) couchincludedir = $(couchlibdir)/include couchebindir = $(couchlibdir)/ebin -couchinclude_DATA = couch_db.hrl +couchinclude_DATA = couch_db.hrl couch_js_functions.hrl couchebin_DATA = $(compiled_files) # dist_devdoc_DATA = $(doc_base) $(doc_modules) @@ -29,7 +29,9 @@ CLEANFILES = $(compiled_files) $(doc_base) source_files = \ couch.erl \ couch_app.erl \ + couch_auth_cache.erl \ couch_btree.erl \ + couch_changes.erl \ couch_config.erl \ couch_config_writer.erl \ couch_db.erl \ @@ -48,6 +50,7 @@ source_files = \ couch_httpd_show.erl \ couch_httpd_view.erl \ couch_httpd_misc_handlers.erl \ + couch_httpd_rewrite.erl \ couch_httpd_stats_handlers.erl \ couch_key_tree.erl \ couch_log.erl \ @@ -78,13 +81,15 @@ source_files = \ couch_db_updater.erl \ couch_work_queue.erl -EXTRA_DIST = $(source_files) couch_db.hrl +EXTRA_DIST = $(source_files) couch_db.hrl couch_js_functions.hrl compiled_files = \ couch.app \ couch.beam \ couch_app.beam \ + couch_auth_cache.beam \ couch_btree.beam \ + couch_changes.beam \ couch_config.beam \ couch_config_writer.beam \ couch_db.beam \ @@ -103,6 +108,7 @@ compiled_files = \ couch_httpd_show.beam \ couch_httpd_view.beam \ couch_httpd_misc_handlers.beam \ + couch_httpd_rewrite.beam \ couch_httpd_stats_handlers.beam \ couch_key_tree.beam \ couch_log.beam \ @@ -175,7 +181,7 @@ couch.app: couch.app.tpl $@ < $< else couch.app: couch.app.tpl - modules=`find . -name "*.erl" -exec basename {} .erl \; | tr '\n' ',' | sed "s/,$$//"`; \ + modules=`{ find . -name "*.erl" -exec basename {} .erl \; | tr '\n' ','; echo ''; } | sed "s/,$$//"`; \ sed -e "s|%package_name%|@package_name@|g" \ -e "s|%version%|@version@|g" \ -e "s|@modules@|$$modules|g" \ @@ -190,6 +196,6 @@ endif # $(ERL) -noshell -run edoc_run files [\"$<\"] -%.beam: %.erl couch_db.hrl +%.beam: %.erl couch_db.hrl couch_js_functions.hrl $(ERLC) $(ERLC_FLAGS) ${TEST} $<; diff --git a/src/couchdb/couch.app.tpl.in b/src/couchdb/couch.app.tpl.in index fa86d2ec..36b0b34c 100644 --- a/src/couchdb/couch.app.tpl.in +++ b/src/couchdb/couch.app.tpl.in @@ -26,4 +26,4 @@ ]}}, {applications, [kernel, stdlib]}, {included_applications, [crypto, sasl, inets, oauth, ibrowse, mochiweb]} -]}.
\ No newline at end of file +]}. diff --git a/src/couchdb/couch_app.erl b/src/couchdb/couch_app.erl index 1b64434a..232953d9 100644 --- a/src/couchdb/couch_app.erl +++ b/src/couchdb/couch_app.erl @@ -20,7 +20,7 @@ start(_Type, DefaultIniFiles) -> IniFiles = get_ini_files(DefaultIniFiles), - case start_apps([crypto, sasl, inets, oauth, ssl, ibrowse, mochiweb]) of + case start_apps([crypto, public_key, sasl, inets, oauth, ssl, ibrowse, mochiweb]) of ok -> couch_server_sup:start_link(IniFiles); {error, Reason} -> @@ -48,6 +48,9 @@ start_apps([App|Rest]) -> start_apps(Rest); {error, {already_started, App}} -> start_apps(Rest); + {error, _Reason} when App =:= public_key -> + % ignore on R12B5 + start_apps(Rest); {error, _Reason} -> {error, {app_would_not_start, App}} end. diff --git a/src/couchdb/couch_auth_cache.erl b/src/couchdb/couch_auth_cache.erl new file mode 100644 index 00000000..e0715b88 --- /dev/null +++ b/src/couchdb/couch_auth_cache.erl @@ -0,0 +1,419 @@ +% 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(couch_auth_cache). +-behaviour(gen_server). + +% public API +-export([get_user_creds/1]). + +% gen_server API +-export([start_link/0, init/1, handle_call/3, handle_info/2, handle_cast/2]). +-export([code_change/3, terminate/2]). + +-include("couch_db.hrl"). +-include("couch_js_functions.hrl"). + +-define(STATE, auth_state_ets). +-define(BY_USER, auth_by_user_ets). +-define(BY_ATIME, auth_by_atime_ets). + +-record(state, { + max_cache_size = 0, + cache_size = 0, + db_notifier = nil +}). + + +-spec get_user_creds(UserName::string() | binary()) -> + Credentials::list() | nil. + +get_user_creds(UserName) when is_list(UserName) -> + get_user_creds(?l2b(UserName)); + +get_user_creds(UserName) -> + UserCreds = case couch_config:get("admins", ?b2l(UserName)) of + "-hashed-" ++ HashedPwdAndSalt -> + % the name is an admin, now check to see if there is a user doc + % which has a matching name, salt, and password_sha + [HashedPwd, Salt] = string:tokens(HashedPwdAndSalt, ","), + case get_from_cache(UserName) of + nil -> + [{<<"roles">>, [<<"_admin">>]}, + {<<"salt">>, ?l2b(Salt)}, + {<<"password_sha">>, ?l2b(HashedPwd)}]; + UserProps when is_list(UserProps) -> + DocRoles = couch_util:get_value(<<"roles">>, UserProps), + [{<<"roles">>, [<<"_admin">> | DocRoles]}, + {<<"salt">>, ?l2b(Salt)}, + {<<"password_sha">>, ?l2b(HashedPwd)}] + end; + _Else -> + get_from_cache(UserName) + end, + validate_user_creds(UserCreds). + + +get_from_cache(UserName) -> + exec_if_auth_db( + fun(_AuthDb) -> + maybe_refresh_cache(), + case ets:lookup(?BY_USER, UserName) of + [] -> + gen_server:call(?MODULE, {fetch, UserName}, infinity); + [{UserName, {Credentials, _ATime}}] -> + couch_stats_collector:increment({couchdb, auth_cache_hits}), + gen_server:cast(?MODULE, {cache_hit, UserName}), + Credentials + end + end, + nil + ). + + +validate_user_creds(nil) -> + nil; +validate_user_creds(UserCreds) -> + case couch_util:get_value(<<"_conflicts">>, UserCreds) of + undefined -> + ok; + _ConflictList -> + throw({unauthorized, + <<"User document conflicts must be resolved before the document", + " is used for authentication purposes.">> + }) + end, + UserCreds. + + +start_link() -> + gen_server:start_link({local, ?MODULE}, ?MODULE, [], []). + + +init(_) -> + ?STATE = ets:new(?STATE, [set, protected, named_table]), + ?BY_USER = ets:new(?BY_USER, [set, protected, named_table]), + ?BY_ATIME = ets:new(?BY_ATIME, [ordered_set, private, named_table]), + AuthDbName = couch_config:get("couch_httpd_auth", "authentication_db"), + true = ets:insert(?STATE, {auth_db_name, ?l2b(AuthDbName)}), + true = ets:insert(?STATE, {auth_db, open_auth_db()}), + process_flag(trap_exit, true), + ok = couch_config:register( + fun("couch_httpd_auth", "auth_cache_size", SizeList) -> + Size = list_to_integer(SizeList), + ok = gen_server:call(?MODULE, {new_max_cache_size, Size}, infinity) + end + ), + ok = couch_config:register( + fun("couch_httpd_auth", "authentication_db", DbName) -> + ok = gen_server:call(?MODULE, {new_auth_db, ?l2b(DbName)}, infinity) + end + ), + {ok, Notifier} = couch_db_update_notifier:start_link(fun handle_db_event/1), + State = #state{ + db_notifier = Notifier, + max_cache_size = list_to_integer( + couch_config:get("couch_httpd_auth", "auth_cache_size", "50") + ) + }, + {ok, State}. + + +handle_db_event({Event, DbName}) -> + [{auth_db_name, AuthDbName}] = ets:lookup(?STATE, auth_db_name), + case DbName =:= AuthDbName of + true -> + case Event of + deleted -> gen_server:call(?MODULE, auth_db_deleted, infinity); + created -> gen_server:call(?MODULE, auth_db_created, infinity); + compacted -> gen_server:call(?MODULE, auth_db_compacted, infinity); + _Else -> ok + end; + false -> + ok + end. + + +handle_call({new_auth_db, AuthDbName}, _From, State) -> + NewState = clear_cache(State), + true = ets:insert(?STATE, {auth_db_name, AuthDbName}), + true = ets:insert(?STATE, {auth_db, open_auth_db()}), + {reply, ok, NewState}; + +handle_call(auth_db_deleted, _From, State) -> + NewState = clear_cache(State), + true = ets:insert(?STATE, {auth_db, nil}), + {reply, ok, NewState}; + +handle_call(auth_db_created, _From, State) -> + NewState = clear_cache(State), + true = ets:insert(?STATE, {auth_db, open_auth_db()}), + {reply, ok, NewState}; + +handle_call(auth_db_compacted, _From, State) -> + exec_if_auth_db( + fun(AuthDb) -> + true = ets:insert(?STATE, {auth_db, reopen_auth_db(AuthDb)}) + end + ), + {reply, ok, State}; + +handle_call({new_max_cache_size, NewSize}, _From, State) -> + case NewSize >= State#state.cache_size of + true -> + ok; + false -> + lists:foreach( + fun(_) -> + LruTime = ets:last(?BY_ATIME), + [{LruTime, UserName}] = ets:lookup(?BY_ATIME, LruTime), + true = ets:delete(?BY_ATIME, LruTime), + true = ets:delete(?BY_USER, UserName) + end, + lists:seq(1, State#state.cache_size - NewSize) + ) + end, + NewState = State#state{ + max_cache_size = NewSize, + cache_size = lists:min([NewSize, State#state.cache_size]) + }, + {reply, ok, NewState}; + +handle_call({fetch, UserName}, _From, State) -> + {Credentials, NewState} = case ets:lookup(?BY_USER, UserName) of + [{UserName, {Creds, ATime}}] -> + couch_stats_collector:increment({couchdb, auth_cache_hits}), + cache_hit(UserName, Creds, ATime), + {Creds, State}; + [] -> + couch_stats_collector:increment({couchdb, auth_cache_misses}), + Creds = get_user_props_from_db(UserName), + State1 = add_cache_entry(UserName, Creds, erlang:now(), State), + {Creds, State1} + end, + {reply, Credentials, NewState}; + +handle_call(refresh, _From, State) -> + exec_if_auth_db(fun refresh_entries/1), + {reply, ok, State}. + + +handle_cast({cache_hit, UserName}, State) -> + case ets:lookup(?BY_USER, UserName) of + [{UserName, {Credentials, ATime}}] -> + cache_hit(UserName, Credentials, ATime); + _ -> + ok + end, + {noreply, State}. + + +handle_info(_Msg, State) -> + {noreply, State}. + + +terminate(_Reason, #state{db_notifier = Notifier}) -> + couch_db_update_notifier:stop(Notifier), + exec_if_auth_db(fun(AuthDb) -> catch couch_db:close(AuthDb) end), + true = ets:delete(?BY_USER), + true = ets:delete(?BY_ATIME), + true = ets:delete(?STATE). + + +code_change(_OldVsn, State, _Extra) -> + {ok, State}. + + +clear_cache(State) -> + exec_if_auth_db(fun(AuthDb) -> catch couch_db:close(AuthDb) end), + true = ets:delete_all_objects(?BY_USER), + true = ets:delete_all_objects(?BY_ATIME), + State#state{cache_size = 0}. + + +add_cache_entry(UserName, Credentials, ATime, State) -> + case State#state.cache_size >= State#state.max_cache_size of + true -> + free_mru_cache_entry(); + false -> + ok + end, + true = ets:insert(?BY_ATIME, {ATime, UserName}), + true = ets:insert(?BY_USER, {UserName, {Credentials, ATime}}), + State#state{cache_size = couch_util:get_value(size, ets:info(?BY_USER))}. + + +free_mru_cache_entry() -> + case ets:last(?BY_ATIME) of + '$end_of_table' -> + ok; % empty cache + LruTime -> + [{LruTime, UserName}] = ets:lookup(?BY_ATIME, LruTime), + true = ets:delete(?BY_ATIME, LruTime), + true = ets:delete(?BY_USER, UserName) + end. + + +cache_hit(UserName, Credentials, ATime) -> + NewATime = erlang:now(), + true = ets:delete(?BY_ATIME, ATime), + true = ets:insert(?BY_ATIME, {NewATime, UserName}), + true = ets:insert(?BY_USER, {UserName, {Credentials, NewATime}}). + + +refresh_entries(AuthDb) -> + case reopen_auth_db(AuthDb) of + nil -> + ok; + AuthDb2 -> + case AuthDb2#db.update_seq > AuthDb#db.update_seq of + true -> + {ok, _, _} = couch_db:enum_docs_since( + AuthDb2, + AuthDb#db.update_seq, + fun(DocInfo, _, _) -> refresh_entry(AuthDb2, DocInfo) end, + AuthDb#db.update_seq, + [] + ), + true = ets:insert(?STATE, {auth_db, AuthDb2}); + false -> + ok + end + end. + + +refresh_entry(Db, #doc_info{high_seq = DocSeq} = DocInfo) -> + case is_user_doc(DocInfo) of + {true, UserName} -> + case ets:lookup(?BY_USER, UserName) of + [] -> + ok; + [{UserName, {_OldCreds, ATime}}] -> + {ok, Doc} = couch_db:open_doc(Db, DocInfo, [conflicts, deleted]), + NewCreds = user_creds(Doc), + true = ets:insert(?BY_USER, {UserName, {NewCreds, ATime}}) + end; + false -> + ok + end, + {ok, DocSeq}. + + +user_creds(#doc{deleted = true}) -> + nil; +user_creds(#doc{} = Doc) -> + {Creds} = couch_query_servers:json_doc(Doc), + Creds. + + +is_user_doc(#doc_info{id = <<"org.couchdb.user:", UserName/binary>>}) -> + {true, UserName}; +is_user_doc(_) -> + false. + + +maybe_refresh_cache() -> + case cache_needs_refresh() of + true -> + ok = gen_server:call(?MODULE, refresh, infinity); + false -> + ok + end. + + +cache_needs_refresh() -> + exec_if_auth_db( + fun(AuthDb) -> + case reopen_auth_db(AuthDb) of + nil -> + false; + AuthDb2 -> + AuthDb2#db.update_seq > AuthDb#db.update_seq + end + end, + false + ). + + +reopen_auth_db(AuthDb) -> + case (catch couch_db:reopen(AuthDb)) of + {ok, AuthDb2} -> + AuthDb2; + _ -> + nil + end. + + +exec_if_auth_db(Fun) -> + exec_if_auth_db(Fun, ok). + +exec_if_auth_db(Fun, DefRes) -> + case ets:lookup(?STATE, auth_db) of + [{auth_db, #db{} = AuthDb}] -> + Fun(AuthDb); + _ -> + DefRes + end. + + +open_auth_db() -> + [{auth_db_name, DbName}] = ets:lookup(?STATE, auth_db_name), + {ok, AuthDb} = ensure_users_db_exists(DbName, [sys_db]), + AuthDb. + + +get_user_props_from_db(UserName) -> + exec_if_auth_db( + fun(AuthDb) -> + Db = reopen_auth_db(AuthDb), + DocId = <<"org.couchdb.user:", UserName/binary>>, + try + {ok, Doc} = couch_db:open_doc(Db, DocId, [conflicts]), + {DocProps} = couch_query_servers:json_doc(Doc), + DocProps + catch + _:_Error -> + nil + end + end, + nil + ). + +ensure_users_db_exists(DbName, Options) -> + Options1 = [{user_ctx, #user_ctx{roles=[<<"_admin">>]}} | Options], + case couch_db:open(DbName, Options1) of + {ok, Db} -> + ensure_auth_ddoc_exists(Db, <<"_design/_auth">>), + {ok, Db}; + _Error -> + {ok, Db} = couch_db:create(DbName, Options1), + ok = ensure_auth_ddoc_exists(Db, <<"_design/_auth">>), + {ok, Db} + end. + +ensure_auth_ddoc_exists(Db, DDocId) -> + case couch_db:open_doc(Db, DDocId) of + {not_found, _Reason} -> + {ok, AuthDesign} = auth_design_doc(DDocId), + {ok, _Rev} = couch_db:update_doc(Db, AuthDesign, []); + _ -> + ok + end, + ok. + +auth_design_doc(DocId) -> + DocProps = [ + {<<"_id">>, DocId}, + {<<"language">>,<<"javascript">>}, + {<<"validate_doc_update">>, ?AUTH_DB_DOC_VALIDATE_FUNCTION} + ], + {ok, couch_doc:from_json_obj({DocProps})}. diff --git a/src/couchdb/couch_btree.erl b/src/couchdb/couch_btree.erl index 73d50805..0e47bac7 100644 --- a/src/couchdb/couch_btree.erl +++ b/src/couchdb/couch_btree.erl @@ -13,7 +13,7 @@ -module(couch_btree). -export([open/2, open/3, query_modify/4, add/2, add_remove/3]). --export([fold/4, full_reduce/1, final_reduce/2,foldl/3,foldl/4]). +-export([fold/4, full_reduce/1, final_reduce/2, foldl/3, foldl/4]). -export([fold_reduce/4, lookup/2, get_state/1, set_options/2]). -define(CHUNK_THRESHOLD, 16#4ff). @@ -70,10 +70,10 @@ final_reduce(Reduce, {KVs, Reductions}) -> final_reduce(Reduce, {[], [Red | Reductions]}). fold_reduce(#btree{root=Root}=Bt, Fun, Acc, Options) -> - Dir = proplists:get_value(dir, Options, fwd), - StartKey = proplists:get_value(start_key, Options), - EndKey = proplists:get_value(end_key, Options), - KeyGroupFun = proplists:get_value(key_group_fun, Options, fun(_,_) -> true end), + Dir = couch_util:get_value(dir, Options, fwd), + StartKey = couch_util:get_value(start_key, Options), + EndKey = couch_util:get_value(end_key, Options), + KeyGroupFun = couch_util:get_value(key_group_fun, Options, fun(_,_) -> true end), {StartKey2, EndKey2} = case Dir of rev -> {EndKey, StartKey}; @@ -107,9 +107,9 @@ convert_fun_arity(Fun) when is_function(Fun, 3) -> Fun. % Already arity 3 make_key_in_end_range_function(#btree{less=Less}, fwd, Options) -> - case proplists:get_value(end_key_gt, Options) of + case couch_util:get_value(end_key_gt, Options) of undefined -> - case proplists:get_value(end_key, Options) of + case couch_util:get_value(end_key, Options) of undefined -> fun(_Key) -> true end; LastKey -> @@ -119,9 +119,9 @@ make_key_in_end_range_function(#btree{less=Less}, fwd, Options) -> fun(Key) -> Less(Key, EndKey) end end; make_key_in_end_range_function(#btree{less=Less}, rev, Options) -> - case proplists:get_value(end_key_gt, Options) of + case couch_util:get_value(end_key_gt, Options) of undefined -> - case proplists:get_value(end_key, Options) of + case couch_util:get_value(end_key, Options) of undefined -> fun(_Key) -> true end; LastKey -> @@ -142,15 +142,15 @@ foldl(Bt, Fun, Acc, Options) -> fold(#btree{root=nil}, _Fun, Acc, _Options) -> {ok, {[], []}, Acc}; fold(#btree{root=Root}=Bt, Fun, Acc, Options) -> - Dir = proplists:get_value(dir, Options, fwd), + Dir = couch_util:get_value(dir, Options, fwd), InRange = make_key_in_end_range_function(Bt, Dir, Options), Result = - case proplists:get_value(start_key, Options) of + case couch_util:get_value(start_key, Options) of undefined -> - stream_node(Bt, [], Bt#btree.root, InRange, Dir, + stream_node(Bt, [], Bt#btree.root, InRange, Dir, convert_fun_arity(Fun), Acc); StartKey -> - stream_node(Bt, [], Bt#btree.root, StartKey, InRange, Dir, + stream_node(Bt, [], Bt#btree.root, StartKey, InRange, Dir, convert_fun_arity(Fun), Acc) end, case Result of @@ -203,8 +203,7 @@ lookup(#btree{root=Root, less=Less}=Bt, Keys) -> % We want to return the results in the same order as the keys were input % but we may have changed the order when we sorted. So we need to put the % order back into the results. - KeyDict = dict:from_list(SortedResults), - [dict:fetch(Key, KeyDict) || Key <- Keys]. + couch_util:reorder_results(Keys, SortedResults). lookup(_Bt, nil, Keys) -> {ok, [{Key, not_found} || Key <- Keys]}; @@ -271,29 +270,26 @@ complete_root(Bt, KPs) -> % written. Plus with the "case byte_size(term_to_binary(InList)) of" code % it's probably really inefficient. -% dialyzer says this pattern is never matched -% chunkify(_Bt, []) -> -% []; -chunkify(Bt, InList) -> +chunkify(InList) -> case byte_size(term_to_binary(InList)) of Size when Size > ?CHUNK_THRESHOLD -> NumberOfChunksLikely = ((Size div ?CHUNK_THRESHOLD) + 1), ChunkThreshold = Size div NumberOfChunksLikely, - chunkify(Bt, InList, ChunkThreshold, [], 0, []); + chunkify(InList, ChunkThreshold, [], 0, []); _Else -> [InList] end. -chunkify(_Bt, [], _ChunkThreshold, [], 0, OutputChunks) -> +chunkify([], _ChunkThreshold, [], 0, OutputChunks) -> lists:reverse(OutputChunks); -chunkify(_Bt, [], _ChunkThreshold, OutList, _OutListSize, OutputChunks) -> +chunkify([], _ChunkThreshold, OutList, _OutListSize, OutputChunks) -> lists:reverse([lists:reverse(OutList) | OutputChunks]); -chunkify(Bt, [InElement | RestInList], ChunkThreshold, OutList, OutListSize, OutputChunks) -> +chunkify([InElement | RestInList], ChunkThreshold, OutList, OutListSize, OutputChunks) -> case byte_size(term_to_binary(InElement)) of Size when (Size + OutListSize) > ChunkThreshold andalso OutList /= [] -> - chunkify(Bt, RestInList, ChunkThreshold, [], 0, [lists:reverse([InElement | OutList]) | OutputChunks]); + chunkify(RestInList, ChunkThreshold, [], 0, [lists:reverse([InElement | OutList]) | OutputChunks]); Size -> - chunkify(Bt, RestInList, ChunkThreshold, [InElement | OutList], OutListSize + Size, OutputChunks) + chunkify(RestInList, ChunkThreshold, [InElement | OutList], OutListSize + Size, OutputChunks) end. modify_node(Bt, RootPointerInfo, Actions, QueryOutput) -> @@ -336,7 +332,7 @@ get_node(#btree{fd = Fd}, NodePos) -> write_node(Bt, NodeType, NodeList) -> % split up nodes into smaller sizes - NodeListList = chunkify(Bt, NodeList), + NodeListList = chunkify(NodeList), % now write out each chunk and return the KeyPointer pairs for those nodes ResultList = [ begin diff --git a/src/couchdb/couch_changes.erl b/src/couchdb/couch_changes.erl new file mode 100644 index 00000000..3a5bc4f8 --- /dev/null +++ b/src/couchdb/couch_changes.erl @@ -0,0 +1,270 @@ +% 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(couch_changes). +-include("couch_db.hrl"). + +-export([handle_changes/3]). + +%% @type Req -> #httpd{} | {json_req, JsonObj()} +handle_changes(#changes_args{style=Style}=Args1, Req, Db) -> + Args = Args1#changes_args{filter= + make_filter_fun(Args1#changes_args.filter, Style, Req, Db)}, + StartSeq = case Args#changes_args.dir of + rev -> + couch_db:get_update_seq(Db); + fwd -> + Args#changes_args.since + end, + if Args#changes_args.feed == "continuous" orelse + Args#changes_args.feed == "longpoll" -> + fun(Callback) -> + Self = self(), + {ok, Notify} = couch_db_update_notifier:start_link( + fun({_, DbName}) when DbName == Db#db.name -> + Self ! db_updated; + (_) -> + ok + end + ), + start_sending_changes(Callback, Args#changes_args.feed), + {Timeout, TimeoutFun} = get_changes_timeout(Args, Callback), + try + keep_sending_changes( + Args, + Callback, + Db, + StartSeq, + <<"">>, + Timeout, + TimeoutFun + ) + after + couch_db_update_notifier:stop(Notify), + get_rest_db_updated() % clean out any remaining update messages + end + end; + true -> + fun(Callback) -> + start_sending_changes(Callback, Args#changes_args.feed), + {ok, {_, LastSeq, _Prepend, _, _, _, _, _}} = + send_changes( + Args#changes_args{feed="normal"}, + Callback, + Db, + StartSeq, + <<"">> + ), + end_sending_changes(Callback, LastSeq, Args#changes_args.feed) + end + end. + +%% @type Req -> #httpd{} | {json_req, JsonObj()} +make_filter_fun(FilterName, Style, Req, Db) -> + case [list_to_binary(couch_httpd:unquote(Part)) + || Part <- string:tokens(FilterName, "/")] of + [] -> + fun(#doc_info{revs=[#rev_info{rev=Rev}|_]=Revs}) -> + case Style of + main_only -> + [{[{<<"rev">>, couch_doc:rev_to_str(Rev)}]}]; + all_docs -> + [{[{<<"rev">>, couch_doc:rev_to_str(R)}]} + || #rev_info{rev=R} <- Revs] + end + end; + [DName, FName] -> + DesignId = <<"_design/", DName/binary>>, + DDoc = couch_httpd_db:couch_doc_open(Db, DesignId, nil, []), + % validate that the ddoc has the filter fun + #doc{body={Props}} = DDoc, + couch_util:get_nested_json_value({Props}, [<<"filters">>, FName]), + fun(DocInfo) -> + DocInfos = + case Style of + main_only -> + [DocInfo]; + all_docs -> + [DocInfo#doc_info{revs=[Rev]}|| Rev <- DocInfo#doc_info.revs] + end, + Docs = [Doc || {ok, Doc} <- [ + couch_db:open_doc(Db, DocInfo2, [deleted, conflicts]) + || DocInfo2 <- DocInfos]], + {ok, Passes} = couch_query_servers:filter_docs( + Req, Db, DDoc, FName, Docs + ), + [{[{<<"rev">>, couch_doc:rev_to_str({RevPos,RevId})}]} + || {Pass, #doc{revs={RevPos,[RevId|_]}}} + <- lists:zip(Passes, Docs), Pass == true] + end; + _Else -> + throw({bad_request, + "filter parameter must be of the form `designname/filtername`"}) + end. + +get_changes_timeout(Args, Callback) -> + #changes_args{ + heartbeat = Heartbeat, + timeout = Timeout, + feed = ResponseType + } = Args, + DefaultTimeout = list_to_integer( + couch_config:get("httpd", "changes_timeout", "60000") + ), + case Heartbeat of + undefined -> + case Timeout of + undefined -> + {DefaultTimeout, fun() -> stop end}; + infinity -> + {infinity, fun() -> stop end}; + _ -> + {lists:min([DefaultTimeout, Timeout]), fun() -> stop end} + end; + true -> + {DefaultTimeout, fun() -> Callback(timeout, ResponseType), ok end}; + _ -> + {lists:min([DefaultTimeout, Heartbeat]), + fun() -> Callback(timeout, ResponseType), ok end} + end. + +start_sending_changes(_Callback, "continuous") -> + ok; +start_sending_changes(Callback, ResponseType) -> + Callback(start, ResponseType). + +send_changes(Args, Callback, Db, StartSeq, Prepend) -> + #changes_args{ + style = Style, + include_docs = IncludeDocs, + limit = Limit, + feed = ResponseType, + dir = Dir, + filter = FilterFun + } = Args, + couch_db:changes_since( + Db, + Style, + StartSeq, + fun changes_enumerator/2, + [{dir, Dir}], + {Db, StartSeq, Prepend, FilterFun, Callback, ResponseType, Limit, + IncludeDocs} + ). + +keep_sending_changes(Args, Callback, Db, StartSeq, Prepend, Timeout, + TimeoutFun) -> + #changes_args{ + feed = ResponseType, + limit = Limit + } = Args, + % ?LOG_INFO("send_changes start ~p",[StartSeq]), + {ok, {_, EndSeq, Prepend2, _, _, _, NewLimit, _}} = send_changes( + Args#changes_args{dir=fwd}, Callback, Db, StartSeq, Prepend + ), + % ?LOG_INFO("send_changes last ~p",[EndSeq]), + couch_db:close(Db), + if Limit > NewLimit, ResponseType == "longpoll" -> + end_sending_changes(Callback, EndSeq, ResponseType); + true -> + case wait_db_updated(Timeout, TimeoutFun) of + updated -> + % ?LOG_INFO("wait_db_updated updated ~p",[{Db#db.name, EndSeq}]), + case couch_db:open(Db#db.name, [{user_ctx, Db#db.user_ctx}]) of + {ok, Db2} -> + keep_sending_changes( + Args#changes_args{limit=NewLimit}, + Callback, + Db2, + EndSeq, + Prepend2, + Timeout, + TimeoutFun + ); + _Else -> + end_sending_changes(Callback, EndSeq, ResponseType) + end; + stop -> + % ?LOG_INFO("wait_db_updated stop ~p",[{Db#db.name, EndSeq}]), + end_sending_changes(Callback, EndSeq, ResponseType) + end + end. + +end_sending_changes(Callback, EndSeq, ResponseType) -> + Callback({stop, EndSeq}, ResponseType). + +changes_enumerator(DocInfo, {Db, _, _, FilterFun, Callback, "continuous", + Limit, IncludeDocs}) -> + + #doc_info{id=Id, high_seq=Seq, + revs=[#rev_info{deleted=Del,rev=Rev}|_]} = DocInfo, + Results0 = FilterFun(DocInfo), + Results = [Result || Result <- Results0, Result /= null], + Go = if Limit =< 1 -> stop; true -> ok end, + case Results of + [] -> + {Go, {Db, Seq, nil, FilterFun, Callback, "continuous", Limit, + IncludeDocs} + }; + _ -> + ChangesRow = changes_row(Db, Seq, Id, Del, Results, Rev, IncludeDocs), + Callback({change, ChangesRow, <<"">>}, "continuous"), + {Go, {Db, Seq, nil, FilterFun, Callback, "continuous", Limit - 1, + IncludeDocs} + } + end; +changes_enumerator(DocInfo, {Db, _, Prepend, FilterFun, Callback, ResponseType, + Limit, IncludeDocs}) -> + + #doc_info{id=Id, high_seq=Seq, revs=[#rev_info{deleted=Del,rev=Rev}|_]} + = DocInfo, + Results0 = FilterFun(DocInfo), + Results = [Result || Result <- Results0, Result /= null], + Go = if Limit =< 1 -> stop; true -> ok end, + case Results of + [] -> + {Go, {Db, Seq, Prepend, FilterFun, Callback, ResponseType, Limit, + IncludeDocs} + }; + _ -> + ChangesRow = changes_row(Db, Seq, Id, Del, Results, Rev, IncludeDocs), + Callback({change, ChangesRow, Prepend}, ResponseType), + {Go, {Db, Seq, <<",\n">>, FilterFun, Callback, ResponseType, Limit - 1, + IncludeDocs} + } + end. + + +changes_row(Db, Seq, Id, Del, Results, Rev, true) -> + {[{<<"seq">>, Seq}, {<<"id">>, Id}, {<<"changes">>, Results}] ++ + deleted_item(Del) ++ couch_httpd_view:doc_member(Db, {Id, Rev})}; +changes_row(_, Seq, Id, Del, Results, _, false) -> + {[{<<"seq">>, Seq}, {<<"id">>, Id}, {<<"changes">>, Results}] ++ + deleted_item(Del)}. + +deleted_item(true) -> [{deleted, true}]; +deleted_item(_) -> []. + +% waits for a db_updated msg, if there are multiple msgs, collects them. +wait_db_updated(Timeout, TimeoutFun) -> + receive db_updated -> get_rest_db_updated() + after Timeout -> + case TimeoutFun() of + ok -> wait_db_updated(Timeout, TimeoutFun); + stop -> stop + end + end. + +get_rest_db_updated() -> + receive db_updated -> get_rest_db_updated() + after 0 -> updated + end. diff --git a/src/couchdb/couch_config.erl b/src/couchdb/couch_config.erl index d8473e08..be53e3a3 100644 --- a/src/couchdb/couch_config.erl +++ b/src/couchdb/couch_config.erl @@ -44,7 +44,7 @@ stop() -> all() -> - lists:sort(gen_server:call(?MODULE, all)). + lists:sort(gen_server:call(?MODULE, all, infinity)). get(Section) when is_binary(Section) -> @@ -111,7 +111,7 @@ terminate(_Reason, _State) -> handle_call(all, _From, Config) -> Resp = lists:sort((ets:tab2list(?MODULE))), {reply, Resp, Config}; -handle_call({set, Sec, Key, Val, Persist}, _From, Config) -> +handle_call({set, Sec, Key, Val, Persist}, From, Config) -> true = ets:insert(?MODULE, {{Sec, Key}, Val}), case {Persist, Config#config.write_filename} of {true, undefined} -> @@ -121,9 +121,12 @@ handle_call({set, Sec, Key, Val, Persist}, _From, Config) -> _ -> ok end, - [catch F(Sec, Key, Val, Persist) || {_Pid, F} <- Config#config.notify_funs], - {reply, ok, Config}; -handle_call({delete, Sec, Key, Persist}, _From, Config) -> + spawn_link(fun() -> + [catch F(Sec, Key, Val, Persist) || {_Pid, F} <- Config#config.notify_funs], + gen_server:reply(From, ok) + end), + {noreply, Config}; +handle_call({delete, Sec, Key, Persist}, From, Config) -> true = ets:delete(?MODULE, {Sec,Key}), case {Persist, Config#config.write_filename} of {true, undefined} -> @@ -133,8 +136,11 @@ handle_call({delete, Sec, Key, Persist}, _From, Config) -> _ -> ok end, - [catch F(Sec, Key, deleted, Persist) || {_Pid, F} <- Config#config.notify_funs], - {reply, ok, Config}; + spawn_link(fun() -> + [catch F(Sec, Key, deleted, Persist) || {_Pid, F} <- Config#config.notify_funs], + gen_server:reply(From, ok) + end), + {noreply, Config}; handle_call({register, Fun, Pid}, _From, #config{notify_funs=PidFuns}=Config) -> erlang:monitor(process, Pid), % convert 1 and 2 arity to 3 arity @@ -194,8 +200,28 @@ parse_ini_file(IniFile) -> {AccSectionName, AccValues}; Line2 -> case re:split(Line2, "\s?=\s?", [{return, list}]) of - [_SingleElement] -> % no "=" found, ignore this line - {AccSectionName, AccValues}; + [Value] -> + MultiLineValuePart = case re:run(Line, "^ \\S", []) of + {match, _} -> + true; + _ -> + false + end, + case {MultiLineValuePart, AccValues} of + {true, [{{_, ValueName}, PrevValue} | AccValuesRest]} -> + % remove comment + case re:split(Value, " ;|\t;", [{return, list}]) of + [[]] -> + % empty line + {AccSectionName, AccValues}; + [LineValue | _Rest] -> + E = {{AccSectionName, ValueName}, + PrevValue ++ " " ++ LineValue}, + {AccSectionName, [E | AccValuesRest]} + end; + _ -> + {AccSectionName, AccValues} + end; [""|_LineValues] -> % line begins with "=", ignore {AccSectionName, AccValues}; [ValueName|LineValues] -> % yeehaw, got a line! @@ -205,7 +231,7 @@ parse_ini_file(IniFile) -> [[]] -> % empty line means delete this key ets:delete(?MODULE, {AccSectionName, ValueName}), - {AccSectionName, AccValues}; + {AccSectionName, AccValues}; [LineValue | _Rest] -> {AccSectionName, [{{AccSectionName, ValueName}, LineValue} | AccValues]} diff --git a/src/couchdb/couch_db.erl b/src/couchdb/couch_db.erl index 79e00ff8..27a3953b 100644 --- a/src/couchdb/couch_db.erl +++ b/src/couchdb/couch_db.erl @@ -13,18 +13,21 @@ -module(couch_db). -behaviour(gen_server). --export([open/2,close/1,create/2,start_compact/1,get_db_info/1,get_design_docs/1]). +-export([open/2,open_int/2,close/1,create/2,start_compact/1,get_db_info/1,get_design_docs/1]). -export([open_ref_counted/2,is_idle/1,monitor/1,count_changes_since/2]). -export([update_doc/3,update_doc/4,update_docs/4,update_docs/2,update_docs/3,delete_doc/3]). -export([get_doc_info/2,open_doc/2,open_doc/3,open_doc_revs/4]). --export([set_revs_limit/2,get_revs_limit/1,register_update_notifier/3]). +-export([set_revs_limit/2,get_revs_limit/1]). -export([get_missing_revs/2,name/1,doc_to_tree/1,get_update_seq/1,get_committed_update_seq/1]). -export([enum_docs/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,open_doc_int/3,set_admins/2,get_admins/1,ensure_full_commit/1]). +-export([start_link/3,open_doc_int/3,ensure_full_commit/1]). +-export([set_security/2,get_security/1]). -export([init/1,terminate/2,handle_call/3,handle_cast/2,code_change/3,handle_info/2]). -export([changes_since/5,changes_since/6,read_doc/2,new_revid/1]). +-export([check_is_admin/1, check_is_reader/1]). +-export([reopen/1]). -include("couch_db.hrl"). @@ -63,9 +66,39 @@ open_db_file(Filepath, Options) -> create(DbName, Options) -> couch_server:create(DbName, Options). -open(DbName, Options) -> +% this is for opening a database for internal purposes like the replicator +% or the view indexer. it never throws a reader error. +open_int(DbName, Options) -> couch_server:open(DbName, Options). +% this should be called anytime an http request opens the database. +% it ensures that the http userCtx is a valid reader +open(DbName, Options) -> + case couch_server:open(DbName, Options) of + {ok, Db} -> + try + check_is_reader(Db), + {ok, Db} + catch + throw:Error -> + close(Db), + throw(Error) + end; + Else -> Else + end. + +reopen(#db{main_pid = Pid, fd_ref_counter = OldRefCntr, user_ctx = UserCtx}) -> + {ok, #db{fd_ref_counter = NewRefCntr} = NewDb} = + gen_server:call(Pid, get_db, infinity), + case NewRefCntr =:= OldRefCntr of + true -> + ok; + false -> + couch_ref_counter:add(NewRefCntr), + couch_ref_counter:drop(OldRefCntr) + end, + {ok, NewDb#db{user_ctx = UserCtx}}. + ensure_full_commit(#db{update_pid=UpdatePid,instance_start_time=StartTime}) -> ok = gen_server:call(UpdatePid, full_commit, infinity), {ok, StartTime}. @@ -73,9 +106,8 @@ ensure_full_commit(#db{update_pid=UpdatePid,instance_start_time=StartTime}) -> close(#db{fd_ref_counter=RefCntr}) -> couch_ref_counter:drop(RefCntr). -open_ref_counted(MainPid, UserCtx) -> - {ok, Db} = gen_server:call(MainPid, {open_ref_count, self()}), - {ok, Db#db{user_ctx=UserCtx}}. +open_ref_counted(MainPid, OpenedPid) -> + gen_server:call(MainPid, {open_ref_count, OpenedPid}). is_idle(MainPid) -> gen_server:call(MainPid, is_idle). @@ -83,11 +115,8 @@ is_idle(MainPid) -> monitor(#db{main_pid=MainPid}) -> erlang:monitor(process, MainPid). -register_update_notifier(#db{main_pid=Pid}, Seq, Fun) -> - gen_server:call(Pid, {register_update_notifier, Seq, Fun}). - start_compact(#db{update_pid=Pid}) -> - gen_server:cast(Pid, start_compact). + gen_server:call(Pid, start_compact). delete_doc(Db, Id, Revisions) -> DeletedDocs = [#doc{id=Id, revs=[Rev], deleted=true} || Rev <- Revisions], @@ -98,23 +127,52 @@ open_doc(Db, IdOrDocInfo) -> open_doc(Db, IdOrDocInfo, []). open_doc(Db, Id, Options) -> - couch_stats_collector:increment({couchdb, database_reads}), + increment_stat(Db, {couchdb, database_reads}), case open_doc_int(Db, Id, Options) of {ok, #doc{deleted=true}=Doc} -> case lists:member(deleted, Options) of true -> - {ok, Doc}; + apply_open_options({ok, Doc},Options); false -> {not_found, deleted} end; Else -> - Else + apply_open_options(Else,Options) + end. + +apply_open_options({ok, Doc},Options) -> + apply_open_options2(Doc,Options); +apply_open_options(Else,_Options) -> + Else. + +apply_open_options2(Doc,[]) -> + {ok, Doc}; +apply_open_options2(#doc{atts=Atts,revs=Revs}=Doc, + [{atts_since, PossibleAncestors}|Rest]) -> + RevPos = find_ancestor_rev_pos(Revs, PossibleAncestors), + apply_open_options2(Doc#doc{atts=[A#att{data= + if AttPos>RevPos -> Data; true -> stub end} + || #att{revpos=AttPos,data=Data}=A <- Atts]}, Rest); +apply_open_options2(Doc,[_|Rest]) -> + apply_open_options2(Doc,Rest). + + +find_ancestor_rev_pos({_, []}, _AttsSinceRevs) -> + 0; +find_ancestor_rev_pos(_DocRevs, []) -> + 0; +find_ancestor_rev_pos({RevPos, [RevId|Rest]}, AttsSinceRevs) -> + case lists:member({RevPos, RevId}, AttsSinceRevs) of + true -> + RevPos; + false -> + find_ancestor_rev_pos({RevPos - 1, Rest}, AttsSinceRevs) end. open_doc_revs(Db, Id, Revs, Options) -> - couch_stats_collector:increment({couchdb, database_reads}), - [Result] = open_doc_revs_int(Db, [{Id, Revs}], Options), - Result. + increment_stat(Db, {couchdb, database_reads}), + [{ok, Results}] = open_doc_revs_int(Db, [{Id, Revs}], Options), + {ok, [apply_open_options(Result, Options) || Result <- Results]}. % Each returned result is a list of tuples: % {Id, MissingRevs, PossibleAncestors} @@ -135,9 +193,9 @@ find_missing([{Id, Revs}|RestIdRevs], [{ok, FullInfo} | RestLookupInfo]) -> % Find the revs that are possible parents of this rev PossibleAncestors = lists:foldl(fun({LeafPos, LeafRevId}, Acc) -> - % this leaf is a "possible ancenstor" of the missing + % this leaf is a "possible ancenstor" of the missing % revs if this LeafPos lessthan any of the missing revs - case lists:any(fun({MissingPos, _}) -> + case lists:any(fun({MissingPos, _}) -> LeafPos < MissingPos end, MissingRevs) of true -> [{LeafPos, LeafRevId} | Acc]; @@ -194,7 +252,8 @@ get_db_info(Db) -> update_seq=SeqNum, name=Name, fulldocinfo_by_id_btree=FullDocBtree, - instance_start_time=StartTime} = Db, + instance_start_time=StartTime, + committed_update_seq=CommittedUpdateSeq} = Db, {ok, Size} = couch_file:bytes(Fd), {ok, {Count, DelCount}} = couch_btree:full_reduce(FullDocBtree), InfoList = [ @@ -206,7 +265,8 @@ get_db_info(Db) -> {compact_running, Compactor/=nil}, {disk_size, Size}, {instance_start_time, StartTime}, - {disk_format_version, DiskVersion} + {disk_format_version, DiskVersion}, + {committed_update_seq, CommittedUpdateSeq} ], {ok, InfoList}. @@ -221,22 +281,88 @@ get_design_docs(#db{fulldocinfo_by_id_btree=Btree}=Db) -> [], [{start_key, <<"_design/">>}, {end_key_gt, <<"_design0">>}]), {ok, Docs}. -check_is_admin(#db{admins=Admins, user_ctx=#user_ctx{name=Name,roles=Roles}}) -> - DbAdmins = [<<"_admin">> | Admins], - case DbAdmins -- [Name | Roles] of - DbAdmins -> % same list, not an admin - throw({unauthorized, <<"You are not a db or server admin.">>}); +check_is_admin(#db{user_ctx=#user_ctx{name=Name,roles=Roles}}=Db) -> + {Admins} = get_admins(Db), + AdminRoles = [<<"_admin">> | couch_util:get_value(<<"roles">>, Admins, [])], + AdminNames = couch_util:get_value(<<"names">>, Admins,[]), + case AdminRoles -- Roles of + AdminRoles -> % same list, not an admin role + case AdminNames -- [Name] of + AdminNames -> % same names, not an admin + throw({unauthorized, <<"You are not a db or server admin.">>}); + _ -> + ok + end; _ -> ok end. -get_admins(#db{admins=Admins}) -> - Admins. +check_is_reader(#db{user_ctx=#user_ctx{name=Name,roles=Roles}=UserCtx}=Db) -> + case (catch check_is_admin(Db)) of + ok -> ok; + _ -> + {Readers} = get_readers(Db), + ReaderRoles = couch_util:get_value(<<"roles">>, Readers,[]), + WithAdminRoles = [<<"_admin">> | ReaderRoles], + ReaderNames = couch_util:get_value(<<"names">>, Readers,[]), + case ReaderRoles ++ ReaderNames of + [] -> ok; % no readers == public access + _Else -> + case WithAdminRoles -- Roles of + WithAdminRoles -> % same list, not an reader role + case ReaderNames -- [Name] of + ReaderNames -> % same names, not a reader + ?LOG_DEBUG("Not a reader: UserCtx ~p vs Names ~p Roles ~p",[UserCtx, ReaderNames, WithAdminRoles]), + throw({unauthorized, <<"You are not authorized to access this db.">>}); + _ -> + ok + end; + _ -> + ok + end + end + end. + +get_admins(#db{security=SecProps}) -> + couch_util:get_value(<<"admins">>, SecProps, {[]}). + +get_readers(#db{security=SecProps}) -> + couch_util:get_value(<<"readers">>, SecProps, {[]}). -set_admins(#db{update_pid=Pid}=Db, Admins) when is_list(Admins) -> +get_security(#db{security=SecProps}) -> + {SecProps}. + +set_security(#db{update_pid=Pid}=Db, {NewSecProps}) when is_list(NewSecProps) -> check_is_admin(Db), - gen_server:call(Pid, {set_admins, Admins}, infinity). + ok = validate_security_object(NewSecProps), + ok = gen_server:call(Pid, {set_security, NewSecProps}, infinity), + {ok, _} = ensure_full_commit(Db), + ok; +set_security(_, _) -> + throw(bad_request). + +validate_security_object(SecProps) -> + Admins = couch_util:get_value(<<"admins">>, SecProps, {[]}), + Readers = couch_util:get_value(<<"readers">>, SecProps, {[]}), + ok = validate_names_and_roles(Admins), + ok = validate_names_and_roles(Readers), + ok. +% validate user input +validate_names_and_roles({Props}) when is_list(Props) -> + case couch_util:get_value(<<"names">>,Props,[]) of + Ns when is_list(Ns) -> + [throw("names must be a JSON list of strings") ||N <- Ns, not is_binary(N)], + Ns; + _ -> throw("names must be a JSON list of strings") + end, + case couch_util:get_value(<<"roles">>,Props,[]) of + Rs when is_list(Rs) -> + [throw("roles must be a JSON list of strings") ||R <- Rs, not is_binary(R)], + Rs; + _ -> throw("roles must be a JSON list of strings") + end, + ok. get_revs_limit(#db{revs_limit=Limit}) -> Limit. @@ -257,8 +383,14 @@ update_doc(Db, Doc, Options, UpdateType) -> case update_docs(Db, [Doc], Options, UpdateType) of {ok, [{ok, NewRev}]} -> {ok, NewRev}; + {ok, [{{_Id, _Rev}, Error}]} -> + throw(Error); {ok, [Error]} -> - throw(Error) + throw(Error); + {ok, []} -> + % replication success + {Pos, [RevId | _]} = Doc#doc.revs, + {ok, {Pos, RevId}} end. update_docs(Db, Docs) -> @@ -285,18 +417,8 @@ 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>>}, _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 - {unauthorized, <<"You are not a server or database admin.">>}; - false -> - ok - end; +validate_doc_update(#db{}=Db, #doc{id= <<"_design/",_/binary>>}, _GetDiskDocFun) -> + catch check_is_admin(Db); validate_doc_update(#db{validate_doc_funs=[]}, _Doc, _GetDiskDocFun) -> ok; validate_doc_update(_Db, #doc{id= <<"_local/",_/binary>>}, _GetDiskDocFun) -> @@ -304,7 +426,8 @@ validate_doc_update(_Db, #doc{id= <<"_local/",_/binary>>}, _GetDiskDocFun) -> validate_doc_update(Db, Doc, GetDiskDocFun) -> DiskDoc = GetDiskDocFun(), JsonCtx = couch_util:json_user_ctx(Db), - try [case Fun(Doc, DiskDoc, JsonCtx) of + SecObj = get_security(Db), + try [case Fun(Doc, DiskDoc, JsonCtx, SecObj) of ok -> ok; Error -> throw(Error) end || Fun <- Db#db.validate_doc_funs], @@ -352,15 +475,15 @@ prep_and_validate_update(Db, #doc{id=Id,revs={RevStart, Revs}}=Doc, prep_and_validate_updates(_Db, [], [], _AllowConflict, AccPrepped, AccFatalErrors) -> {AccPrepped, AccFatalErrors}; -prep_and_validate_updates(Db, [DocBucket|RestBuckets], [not_found|RestLookups], +prep_and_validate_updates(Db, [DocBucket|RestBuckets], [not_found|RestLookups], AllowConflict, AccPrepped, AccErrors) -> [#doc{id=Id}|_]=DocBucket, % no existing revs are known, {PreppedBucket, AccErrors3} = lists:foldl( - fun(#doc{revs=Revs}=Doc, {AccBucket, AccErrors2}) -> + fun(#doc{revs=Revs}=Doc, {AccBucket, AccErrors2}) -> case couch_doc:has_stubs(Doc) of true -> - couch_doc:merge_doc(Doc, #doc{}); % will throw exception + couch_doc:merge_stubs(Doc, #doc{}); % will throw exception false -> ok end, case Revs of @@ -398,7 +521,7 @@ prep_and_validate_updates(Db, [DocBucket|RestBuckets], end end, {[], AccErrors}, DocBucket), - prep_and_validate_updates(Db, RestBuckets, RestLookups, AllowConflict, + prep_and_validate_updates(Db, RestBuckets, RestLookups, AllowConflict, [PreppedBucket | AccPrepped], AccErrors3). @@ -414,10 +537,10 @@ prep_and_validate_replicated_updates(Db, [Bucket|RestBuckets], [OldInfo|RestOldI case OldInfo of not_found -> {ValidatedBucket, AccErrors3} = lists:foldl( - fun(Doc, {AccPrepped2, AccErrors2}) -> + fun(Doc, {AccPrepped2, AccErrors2}) -> case couch_doc:has_stubs(Doc) of true -> - couch_doc:merge_doc(Doc, #doc{}); % will throw exception + couch_doc:merge_stubs(Doc, #doc{}); % will throw exception false -> ok end, case validate_doc_update(Db, Doc, fun() -> nil end) of @@ -432,7 +555,7 @@ prep_and_validate_replicated_updates(Db, [Bucket|RestBuckets], [OldInfo|RestOldI {ok, #full_doc_info{rev_tree=OldTree}} -> NewRevTree = lists:foldl( fun(NewDoc, AccTree) -> - {NewTree, _} = couch_key_tree:merge(AccTree, [couch_db:doc_to_tree(NewDoc)]), + {NewTree, _} = couch_key_tree:merge(AccTree, couch_db:doc_to_tree(NewDoc)), NewTree end, OldTree, Bucket), @@ -487,7 +610,7 @@ new_revid(#doc{body=Body,revs={OldStart,OldRevs}, ?l2b(integer_to_list(couch_util:rand32())); Atts2 -> OldRev = case OldRevs of [] -> 0; [OldRev0|_] -> OldRev0 end, - erlang:md5(term_to_binary([Deleted, OldStart, OldRev, Body, Atts2])) + couch_util:md5(term_to_binary([Deleted, OldStart, OldRev, Body, Atts2])) end. new_revs([], OutBuckets, IdRevsAcc) -> @@ -515,7 +638,7 @@ check_dup_atts2(_) -> update_docs(Db, Docs, Options, replicated_changes) -> - couch_stats_collector:increment({couchdb, database_writes}), + increment_stat(Db, {couchdb, database_writes}), DocBuckets = group_alike_docs(Docs), case (Db#db.validate_doc_funs /= []) orelse @@ -541,7 +664,7 @@ update_docs(Db, Docs, Options, replicated_changes) -> {ok, DocErrors}; update_docs(Db, Docs, Options, interactive_edit) -> - couch_stats_collector:increment({couchdb, database_writes}), + increment_stat(Db, {couchdb, database_writes}), AllOrNothing = lists:member(all_or_nothing, Options), % go ahead and generate the new revision ids for the documents. % separate out the NonRep documents from the rest of the documents @@ -645,7 +768,7 @@ collect_results(UpdatePid, MRef, ResultsAcc) -> exit(Reason) end. -write_and_commit(#db{update_pid=UpdatePid, user_ctx=Ctx}=Db, DocBuckets, +write_and_commit(#db{update_pid=UpdatePid}=Db, DocBuckets, NonRepDocs, Options0) -> Options = set_commit_option(Options0), MergeConflicts = lists:member(merge_conflicts, Options), @@ -658,7 +781,7 @@ write_and_commit(#db{update_pid=UpdatePid, user_ctx=Ctx}=Db, DocBuckets, retry -> % This can happen if the db file we wrote to was swapped out by % compaction. Retry by reopening the db and writing to the current file - {ok, Db2} = open_ref_counted(Db#db.main_pid, Ctx), + {ok, Db2} = open_ref_counted(Db#db.main_pid, self()), DocBuckets2 = [[doc_flush_atts(Doc, Db2#db.fd) || Doc <- Bucket] || Bucket <- DocBuckets], % We only retry once close(Db2), @@ -693,18 +816,19 @@ flush_att(Fd, #att{data={Fd0, _}}=Att) when Fd0 == Fd -> % already written to our file, nothing to write Att; -flush_att(Fd, #att{data={OtherFd,StreamPointer}, md5=InMd5}=Att) -> - {NewStreamData, Len, Md5} = +flush_att(Fd, #att{data={OtherFd,StreamPointer}, md5=InMd5, + disk_len=InDiskLen} = Att) -> + {NewStreamData, Len, _IdentityLen, Md5, IdentityMd5} = couch_stream:copy_to_new_stream(OtherFd, StreamPointer, Fd), - check_md5(Md5, InMd5), - Att#att{data={Fd, NewStreamData}, md5=Md5, len=Len}; + check_md5(IdentityMd5, InMd5), + Att#att{data={Fd, NewStreamData}, md5=Md5, att_len=Len, disk_len=InDiskLen}; flush_att(Fd, #att{data=Data}=Att) when is_binary(Data) -> with_stream(Fd, Att, fun(OutputStream) -> couch_stream:write(OutputStream, Data) end); -flush_att(Fd, #att{data=Fun,len=undefined}=Att) when is_function(Fun) -> +flush_att(Fd, #att{data=Fun,att_len=undefined}=Att) when is_function(Fun) -> with_stream(Fd, Att, fun(OutputStream) -> % Fun(MaxChunkSize, WriterFun) must call WriterFun % once for each chunk of the attachment, @@ -726,9 +850,9 @@ flush_att(Fd, #att{data=Fun,len=undefined}=Att) when is_function(Fun) -> end, ok) end); -flush_att(Fd, #att{data=Fun,len=Len}=Att) when is_function(Fun) -> +flush_att(Fd, #att{data=Fun,att_len=AttLen}=Att) when is_function(Fun) -> with_stream(Fd, Att, fun(OutputStream) -> - write_streamed_attachment(OutputStream, Fun, Len) + write_streamed_attachment(OutputStream, Fun, AttLen) end). % From RFC 2616 3.6.1 - Chunked Transfer Coding @@ -741,8 +865,17 @@ flush_att(Fd, #att{data=Fun,len=Len}=Att) when is_function(Fun) -> % is present in the request, but there is no Content-MD5 % trailer, we're free to ignore this inconsistency and % pretend that no Content-MD5 exists. -with_stream(Fd, #att{md5=InMd5}=Att, Fun) -> - {ok, OutputStream} = couch_stream:open(Fd), +with_stream(Fd, #att{md5=InMd5,type=Type,encoding=Enc}=Att, Fun) -> + {ok, OutputStream} = case (Enc =:= identity) andalso + couch_util:compressible_att_type(Type) of + true -> + CompLevel = list_to_integer( + couch_config:get("attachments", "compression_level", "0") + ), + couch_stream:open(Fd, gzip, [{compression_level, CompLevel}]); + _ -> + couch_stream:open(Fd) + end, ReqMd5 = case Fun(OutputStream) of {md5, FooterMd5} -> case InMd5 of @@ -752,9 +885,36 @@ with_stream(Fd, #att{md5=InMd5}=Att, Fun) -> _ -> InMd5 end, - {StreamInfo, Len, Md5} = couch_stream:close(OutputStream), - check_md5(Md5, ReqMd5), - Att#att{data={Fd,StreamInfo},len=Len,md5=Md5}. + {StreamInfo, Len, IdentityLen, Md5, IdentityMd5} = + couch_stream:close(OutputStream), + check_md5(IdentityMd5, ReqMd5), + {AttLen, DiskLen, NewEnc} = case Enc of + identity -> + case {Md5, IdentityMd5} of + {Same, Same} -> + {Len, IdentityLen, identity}; + _ -> + {Len, IdentityLen, gzip} + end; + gzip -> + case {Att#att.att_len, Att#att.disk_len} of + {AL, DL} when AL =:= undefined orelse DL =:= undefined -> + % Compressed attachment uploaded through the standalone API. + {Len, Len, gzip}; + {AL, DL} -> + % This case is used for efficient push-replication, where a + % compressed attachment is located in the body of multipart + % content-type request. + {AL, DL, gzip} + end + end, + Att#att{ + data={Fd,StreamInfo}, + att_len=AttLen, + disk_len=DiskLen, + md5=Md5, + encoding=NewEnc + }. write_streamed_attachment(_Stream, _F, 0) -> @@ -779,17 +939,18 @@ changes_since(Db, Style, StartSeq, Fun, Acc) -> changes_since(Db, Style, StartSeq, Fun, Options, Acc) -> Wrapper = fun(DocInfo, _Offset, Acc2) -> #doc_info{revs=Revs} = DocInfo, + DocInfo2 = case Style of main_only -> - Infos = [DocInfo]; + DocInfo; all_docs -> - % make each rev it's own doc info - Infos = [DocInfo#doc_info{revs=[RevInfo]} || - #rev_info{seq=RevSeq}=RevInfo <- Revs, StartSeq < RevSeq] + % remove revs before the seq + DocInfo#doc_info{revs=[RevInfo || + #rev_info{seq=RevSeq}=RevInfo <- Revs, StartSeq < RevSeq]} end, - Fun(Infos, Acc2) + Fun(DocInfo2, Acc2) end, - {ok, _LastReduction, AccOut} = couch_btree:fold(Db#db.docinfo_by_seq_btree, + {ok, _LastReduction, AccOut} = couch_btree:fold(Db#db.docinfo_by_seq_btree, Wrapper, Acc, [{start_key, StartSeq + 1}] ++ Options), {ok, AccOut}. @@ -816,11 +977,17 @@ init({DbName, Filepath, Fd, Options}) -> {ok, UpdaterPid} = gen_server:start_link(couch_db_updater, {self(), DbName, Filepath, Fd, Options}, []), {ok, #db{fd_ref_counter=RefCntr}=Db} = gen_server:call(UpdaterPid, get_db), couch_ref_counter:add(RefCntr), - couch_stats_collector:track_process_count({couchdb, open_databases}), + case lists:member(sys_db, Options) of + true -> + ok; + false -> + couch_stats_collector:track_process_count({couchdb, open_databases}) + end, + process_flag(trap_exit, true), {ok, Db}. -terminate(Reason, _Db) -> - couch_util:terminate_linked(Reason), +terminate(_Reason, Db) -> + couch_util:shutdown_sync(Db#db.update_pid), ok. handle_call({open_ref_count, OpenerPid}, _, #db{fd_ref_counter=RefCntr}=Db) -> @@ -839,7 +1006,9 @@ handle_call({db_updated, NewDb}, _From, #db{fd_ref_counter=OldRefCntr}) -> couch_ref_counter:add(NewRefCntr), couch_ref_counter:drop(OldRefCntr) end, - {reply, ok, NewDb}. + {reply, ok, NewDb}; +handle_call(get_db, _From, Db) -> + {reply, {ok, Db}, Db}. handle_cast(Msg, Db) -> @@ -848,7 +1017,11 @@ handle_cast(Msg, Db) -> code_change(_OldVsn, State, _Extra) -> {ok, State}. - + +handle_info({'EXIT', _Pid, normal}, Db) -> + {noreply, Db}; +handle_info({'EXIT', _Pid, Reason}, Server) -> + {stop, Reason, Server}; handle_info(Msg, Db) -> ?LOG_ERROR("Bad message received for db ~s: ~p", [Db#db.name, Msg]), exit({error, Msg}). @@ -983,17 +1156,39 @@ make_doc(#db{fd=Fd}=Db, Id, Deleted, Bp, RevisionPath) -> {ok, {BodyData0, Atts0}} = read_doc(Db, Bp), {BodyData0, lists:map( - fun({Name,Type,Sp,Len,RevPos,Md5}) -> + fun({Name,Type,Sp,AttLen,DiskLen,RevPos,Md5,Enc}) -> #att{name=Name, type=Type, - len=Len, + att_len=AttLen, + disk_len=DiskLen, + md5=Md5, + revpos=RevPos, + data={Fd,Sp}, + encoding= + case Enc of + true -> + % 0110 UPGRADE CODE + gzip; + false -> + % 0110 UPGRADE CODE + identity; + _ -> + Enc + end + }; + ({Name,Type,Sp,AttLen,RevPos,Md5}) -> + #att{name=Name, + type=Type, + att_len=AttLen, + disk_len=AttLen, md5=Md5, revpos=RevPos, data={Fd,Sp}}; - ({Name,{Type,Sp,Len}}) -> + ({Name,{Type,Sp,AttLen}}) -> #att{name=Name, type=Type, - len=Len, + att_len=AttLen, + disk_len=AttLen, md5= <<>>, revpos=0, data={Fd,Sp}} @@ -1008,4 +1203,7 @@ make_doc(#db{fd=Fd}=Db, Id, Deleted, Bp, RevisionPath) -> }. - +increment_stat(#db{is_sys_db = true}, _Stat) -> + ok; +increment_stat(#db{}, Stat) -> + couch_stats_collector:increment(Stat). diff --git a/src/couchdb/couch_db.hrl b/src/couchdb/couch_db.hrl index 99ef8997..a35745ef 100644 --- a/src/couchdb/couch_db.hrl +++ b/src/couchdb/couch_db.hrl @@ -20,7 +20,6 @@ -define(JSON_ENCODE(V), couch_util:json_encode(V)). -define(JSON_DECODE(V), couch_util:json_decode(V)). --define(b2a(V), list_to_atom(binary_to_list(V))). -define(b2l(V), binary_to_list(V)). -define(l2b(V), list_to_binary(V)). @@ -28,18 +27,23 @@ -define(LOG_DEBUG(Format, Args), case couch_log:debug_on() of - true -> error_logger:info_report(couch_debug, {Format, Args}); + true -> + gen_event:sync_notify(error_logger, + {self(), couch_debug, {Format, Args}}); false -> ok end). -define(LOG_INFO(Format, Args), case couch_log:info_on() of - true -> error_logger:info_report(couch_info, {Format, Args}); + true -> + gen_event:sync_notify(error_logger, + {self(), couch_info, {Format, Args}}); false -> ok end). -define(LOG_ERROR(Format, Args), - error_logger:error_report(couch_error, {Format, Args})). + gen_event:sync_notify(error_logger, + {self(), couch_error, {Format, Args}})). -record(rev_info, @@ -73,7 +77,9 @@ user_ctx, req_body = undefined, design_url_handlers, - auth + auth, + default_fun, + url_handlers }). @@ -99,10 +105,17 @@ { name, type, - len, + att_len, + disk_len, % length of the attachment in its identity form + % (that is, without a content encoding applied to it) + % differs from att_len when encoding /= identity md5= <<>>, revpos=0, - data + data, + encoding=identity % currently supported values are: + % identity, gzip + % additional values to support in the future: + % deflate, compress }). @@ -110,8 +123,7 @@ { name=null, roles=[], - handler, - user_doc + handler }). % This should be updated anytime a header change happens that requires more @@ -124,7 +136,7 @@ % 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, 4). +-define(LATEST_DISK_VERSION, 5). -record(db_header, {disk_version = ?LATEST_DISK_VERSION, @@ -135,7 +147,7 @@ local_docs_btree_state = nil, purge_seq = 0, purged_docs = nil, - admins_ptr = nil, + security_ptr = nil, revs_limit = 1000 }). @@ -155,12 +167,13 @@ name, filepath, validate_doc_funs = [], - admins = [], - admins_ptr = nil, + security = [], + security_ptr = nil, user_ctx = #user_ctx{}, waiting_delayed_commit = nil, revs_limit = 1000, - fsync_options = [] + fsync_options = [], + is_sys_db = false }). @@ -251,7 +264,13 @@ body = nil, options = [ {response_format,binary}, - {inactivity_timeout, 30000} + {inactivity_timeout, 30000}, + {max_sessions, list_to_integer( + couch_config:get("replicator", "max_http_sessions", "10") + )}, + {max_pipeline_size, list_to_integer( + couch_config:get("replicator", "max_http_pipeline_size", "10") + )} ], retries = 10, pause = 500, @@ -260,3 +279,16 @@ % small value used in revision trees to indicate the revision isn't stored -define(REV_MISSING, []). + +-record(changes_args, { + feed = "normal", + dir = fwd, + since = 0, + limit = 1000000000000000, + style = main_only, + heartbeat, + timeout, + filter = "", + include_docs = false +}). + diff --git a/src/couchdb/couch_db_updater.erl b/src/couchdb/couch_db_updater.erl index 7292221a..633ae230 100644 --- a/src/couchdb/couch_db_updater.erl +++ b/src/couchdb/couch_db_updater.erl @@ -20,25 +20,38 @@ init({MainPid, DbName, Filepath, Fd, Options}) -> + process_flag(trap_exit, true), case lists:member(create, Options) of true -> % create a new header and writes it to the file Header = #db_header{}, ok = couch_file:write_header(Fd, Header), % delete any old compaction files that might be hanging around - file:delete(Filepath ++ ".compact"); + RootDir = couch_config:get("couchdb", "database_dir", "."), + couch_file:delete(RootDir, Filepath ++ ".compact"); false -> ok = couch_file:upgrade_old_header(Fd, <<$g, $m, $k, 0>>), % 09 UPGRADE CODE - {ok, Header} = couch_file:read_header(Fd) + case couch_file:read_header(Fd) of + {ok, Header} -> + ok; + no_valid_header -> + % create a new header and writes it to the file + Header = #db_header{}, + ok = couch_file:write_header(Fd, Header), + % delete any old compaction files that might be hanging around + file:delete(Filepath ++ ".compact") + end end, Db = init_db(DbName, Filepath, Fd, Header), Db2 = refresh_validate_doc_funs(Db), - {ok, Db2#db{main_pid=MainPid}}. + {ok, Db2#db{main_pid = MainPid, is_sys_db = lists:member(sys_db, Options)}}. -terminate(Reason, _Srv) -> - couch_util:terminate_linked(Reason), +terminate(_Reason, Db) -> + couch_file:close(Db#db.fd), + couch_util:shutdown_sync(Db#db.compactor_pid), + couch_util:shutdown_sync(Db#db.fd_ref_counter), ok. handle_call(get_db, _From, Db) -> @@ -53,9 +66,9 @@ 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}, _From, Db) -> - {ok, Ptr} = couch_file:append_term(Db#db.fd, NewAdmins), - Db2 = commit_data(Db#db{admins=NewAdmins, admins_ptr=Ptr, +handle_call({set_security, NewSec}, _From, Db) -> + {ok, Ptr} = couch_file:append_term(Db#db.fd, NewSec), + Db2 = commit_data(Db#db{security=NewSec, security_ptr=Ptr, update_seq=Db#db.update_seq+1}), ok = gen_server:call(Db2#db.main_pid, {db_updated, Db2}), {reply, ok, Db2}; @@ -130,21 +143,22 @@ handle_call({purge_docs, IdRevs}, _From, Db) -> ok = gen_server:call(Db2#db.main_pid, {db_updated, Db2}), couch_db_update_notifier:notify({updated, Db#db.name}), - {reply, {ok, Db2#db.update_seq, IdRevsPurged}, Db2}. - - -handle_cast(start_compact, Db) -> + {reply, {ok, (Db2#db.header)#db_header.purge_seq, IdRevsPurged}, Db2}; +handle_call(start_compact, _From, 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(Db) end), Db2 = Db#db{compactor_pid=Pid}, ok = gen_server:call(Db#db.main_pid, {db_updated, Db2}), - {noreply, Db2}; + {reply, ok, Db2}; _ -> % compact currently running, this is a no-op - {noreply, Db} - end; + {reply, ok, Db} + end. + + + handle_cast({compact_done, CompactFilepath}, #db{filepath=Filepath}=Db) -> {ok, NewFd} = couch_file:open(CompactFilepath), {ok, NewHeader} = couch_file:read_header(NewFd), @@ -158,17 +172,25 @@ handle_cast({compact_done, CompactFilepath}, #db{filepath=Filepath}=Db) -> fun(Value, _Offset, Acc) -> {ok, [Value | Acc]} end, []), {ok, NewLocalBtree} = couch_btree:add(NewDb#db.local_docs_btree, LocalDocs), - NewDb2 = commit_data( NewDb#db{local_docs_btree=NewLocalBtree, - main_pid = Db#db.main_pid,filepath = Filepath}), + NewDb2 = commit_data(NewDb#db{ + local_docs_btree = NewLocalBtree, + main_pid = Db#db.main_pid, + filepath = Filepath, + instance_start_time = Db#db.instance_start_time, + revs_limit = Db#db.revs_limit + }), ?LOG_DEBUG("CouchDB swapping files ~s and ~s.", [Filepath, CompactFilepath]), - file:delete(Filepath), + RootDir = couch_config:get("couchdb", "database_dir", "."), + couch_file:delete(RootDir, Filepath), ok = file:rename(CompactFilepath, Filepath), close_db(Db), - ok = gen_server:call(Db#db.main_pid, {db_updated, NewDb2}), + NewDb3 = refresh_validate_doc_funs(NewDb2), + ok = gen_server:call(Db#db.main_pid, {db_updated, NewDb3}, infinity), + couch_db_update_notifier:notify({compacted, NewDb3#db.name}), ?LOG_INFO("Compaction for db \"~s\" completed.", [Db#db.name]), - {noreply, NewDb2#db{compactor_pid=nil}}; + {noreply, NewDb3#db{compactor_pid=nil}}; false -> ?LOG_INFO("Compaction file still behind main file " "(update seq=~p. compact update seq=~p). Retrying.", @@ -180,11 +202,11 @@ handle_cast({compact_done, CompactFilepath}, #db{filepath=Filepath}=Db) -> end. -handle_info({update_docs, Client, GroupedDocs, NonRepDocs, MergeConflicts, +handle_info({update_docs, Client, GroupedDocs, NonRepDocs, MergeConflicts, FullCommit}, Db) -> GroupedDocs2 = [[{Client, D} || D <- DocGroup] || DocGroup <- GroupedDocs], if NonRepDocs == [] -> - {GroupedDocs3, Clients, FullCommit2} = collect_updates(GroupedDocs2, + {GroupedDocs3, Clients, FullCommit2} = collect_updates(GroupedDocs2, [Client], MergeConflicts, FullCommit); true -> GroupedDocs3 = GroupedDocs2, @@ -192,7 +214,7 @@ handle_info({update_docs, Client, GroupedDocs, NonRepDocs, MergeConflicts, Clients = [Client] end, NonRepDocs2 = [{Client, NRDoc} || NRDoc <- NonRepDocs], - try update_docs_int(Db, GroupedDocs3, NonRepDocs2, MergeConflicts, + try update_docs_int(Db, GroupedDocs3, NonRepDocs2, MergeConflicts, FullCommit2) of {ok, Db2} -> ok = gen_server:call(Db#db.main_pid, {db_updated, Db2}), @@ -207,6 +229,9 @@ handle_info({update_docs, Client, GroupedDocs, NonRepDocs, MergeConflicts, [catch(ClientPid ! {retry, self()}) || ClientPid <- Clients], {noreply, Db} end; +handle_info(delayed_commit, #db{waiting_delayed_commit=nil}=Db) -> + %no outstanding delayed commits, ignore + {noreply, Db}; handle_info(delayed_commit, Db) -> case commit_data(Db) of Db -> @@ -214,7 +239,11 @@ handle_info(delayed_commit, Db) -> Db2 -> ok = gen_server:call(Db2#db.main_pid, {db_updated, Db2}), {noreply, Db2} - end. + end; +handle_info({'EXIT', _Pid, normal}, Db) -> + {noreply, Db}; +handle_info({'EXIT', _Pid, Reason}, Db) -> + {stop, Reason, Db}. code_change(_OldVsn, State, _Extra) -> {ok, State}. @@ -243,7 +272,7 @@ collect_updates(GroupedDocsAcc, ClientsAcc, MergeConflicts, FullCommit) -> {update_docs, Client, GroupedDocs, [], MergeConflicts, FullCommit2} -> GroupedDocs2 = [[{Client, Doc} || Doc <- DocGroup] || DocGroup <- GroupedDocs], - GroupedDocsAcc2 = + GroupedDocsAcc2 = merge_updates(GroupedDocsAcc, GroupedDocs2, []), collect_updates(GroupedDocsAcc2, [Client | ClientsAcc], MergeConflicts, (FullCommit or FullCommit2)) @@ -326,7 +355,7 @@ btree_by_seq_reduce(rereduce, Reds) -> simple_upgrade_record(Old, New) when tuple_size(Old) =:= tuple_size(New) -> Old; -simple_upgrade_record(Old, New) -> +simple_upgrade_record(Old, New) when tuple_size(Old) < tuple_size(New) -> OldSz = tuple_size(Old), NewValuesTail = lists:sublist(tuple_to_list(New), OldSz + 1, tuple_size(New) - OldSz), @@ -337,9 +366,10 @@ init_db(DbName, Filepath, Fd, Header0) -> Header1 = simple_upgrade_record(Header0, #db_header{}), Header = case element(2, Header1) of - 1 -> Header1#db_header{unused = 0}; % 0.9 - 2 -> Header1#db_header{unused = 0}; % post 0.9 and pre 0.10 - 3 -> Header1; % post 0.9 and pre 0.10 + 1 -> Header1#db_header{unused = 0, security_ptr = nil}; % 0.9 + 2 -> Header1#db_header{unused = 0, security_ptr = nil}; % post 0.9 and pre 0.10 + 3 -> Header1#db_header{security_ptr = nil}; % post 0.9 and pre 0.10 + 4 -> Header1#db_header{security_ptr = nil}; % 0.10 and pre 0.11 ?LATEST_DISK_VERSION -> Header1; _ -> throw({database_disk_version_error, "Incorrect disk header version"}) end, @@ -362,12 +392,12 @@ init_db(DbName, Filepath, Fd, Header0) -> {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 + case Header#db_header.security_ptr of nil -> - Admins = [], - AdminsPtr = nil; - AdminsPtr -> - {ok, Admins} = couch_file:pread_term(Fd, AdminsPtr) + Security = [], + SecurityPtr = nil; + SecurityPtr -> + {ok, Security} = couch_file:pread_term(Fd, SecurityPtr) end, % convert start time tuple to microsecs and store as a binary string {MegaSecs, Secs, MicroSecs} = now(), @@ -386,8 +416,8 @@ init_db(DbName, Filepath, Fd, Header0) -> update_seq = Header#db_header.update_seq, name = DbName, filepath = Filepath, - admins = Admins, - admins_ptr = AdminsPtr, + security = Security, + security_ptr = SecurityPtr, instance_start_time = StartTime, revs_limit = Header#db_header.revs_limit, fsync_options = FsyncOptions @@ -429,8 +459,9 @@ flush_trees(#db{fd=Fd,header=Header}=Db, case Atts of [] -> []; [#att{data={BinFd, _Sp}} | _ ] when BinFd == Fd -> - [{N,T,P,L,R,M} - || #att{name=N,type=T,data={_,P},md5=M,revpos=R,len=L} + [{N,T,P,AL,DL,R,M,E} + || #att{name=N,type=T,data={_,P},md5=M,revpos=R, + att_len=AL,disk_len=DL,encoding=E} <- Atts]; _ -> % BinFd must not equal our Fd. This can happen when a database @@ -458,16 +489,16 @@ send_result(Client, Id, OriginalRevs, NewResult) -> % used to send a result to the client catch(Client ! {result, self(), {{Id, OriginalRevs}, NewResult}}). -merge_rev_trees(_MergeConflicts, [], [], AccNewInfos, AccRemoveSeqs, AccSeq) -> +merge_rev_trees(_Limit, _Merge, [], [], AccNewInfos, AccRemoveSeqs, AccSeq) -> {ok, lists:reverse(AccNewInfos), AccRemoveSeqs, AccSeq}; -merge_rev_trees(MergeConflicts, [NewDocs|RestDocsList], +merge_rev_trees(Limit, MergeConflicts, [NewDocs|RestDocsList], [OldDocInfo|RestOldInfo], AccNewInfos, AccRemoveSeqs, AccSeq) -> #full_doc_info{id=Id,rev_tree=OldTree,deleted=OldDeleted,update_seq=OldSeq} = OldDocInfo, - NewRevTree = lists:foldl( + NewRevTree0 = lists:foldl( fun({Client, #doc{revs={Pos,[_Rev|PrevRevs]}}=NewDoc}, AccTree) -> if not MergeConflicts -> - case couch_key_tree:merge(AccTree, [couch_db:doc_to_tree(NewDoc)]) of + case couch_key_tree:merge(AccTree, couch_db:doc_to_tree(NewDoc)) of {_NewTree, conflicts} when (not OldDeleted) -> send_result(Client, Id, {Pos-1,PrevRevs}, conflict), AccTree; @@ -492,15 +523,15 @@ merge_rev_trees(MergeConflicts, [NewDocs|RestDocsList], % this means we are recreating a brand new document % into a state that already existed before. % put the rev into a subsequent edit of the deletion - #doc_info{revs=[#rev_info{rev={OldPos,OldRev}}|_]} = + #doc_info{revs=[#rev_info{rev={OldPos,OldRev}}|_]} = couch_doc:to_doc_info(OldDocInfo), NewRevId = couch_db:new_revid( NewDoc#doc{revs={OldPos, [OldRev]}}), NewDoc2 = NewDoc#doc{revs={OldPos + 1, [NewRevId, OldRev]}}, {NewTree2, _} = couch_key_tree:merge(AccTree, - [couch_db:doc_to_tree(NewDoc2)]), + couch_db:doc_to_tree(NewDoc2)), % we changed the rev id, this tells the caller we did - send_result(Client, Id, {Pos-1,PrevRevs}, + send_result(Client, Id, {Pos-1,PrevRevs}, {ok, {OldPos + 1, NewRevId}}), NewTree2; true -> @@ -512,15 +543,16 @@ merge_rev_trees(MergeConflicts, [NewDocs|RestDocsList], end; true -> {NewTree, _} = couch_key_tree:merge(AccTree, - [couch_db:doc_to_tree(NewDoc)]), + couch_db:doc_to_tree(NewDoc)), NewTree - end + end end, OldTree, NewDocs), + NewRevTree = couch_key_tree:stem(NewRevTree0, Limit), if NewRevTree == OldTree -> % nothing changed - merge_rev_trees(MergeConflicts, RestDocsList, RestOldInfo, AccNewInfos, - AccRemoveSeqs, AccSeq); + merge_rev_trees(Limit, MergeConflicts, RestDocsList, RestOldInfo, + AccNewInfos, AccRemoveSeqs, AccSeq); true -> % we have updated the document, give it a new seq # NewInfo = #full_doc_info{id=Id,update_seq=AccSeq+1,rev_tree=NewRevTree}, @@ -528,8 +560,8 @@ merge_rev_trees(MergeConflicts, [NewDocs|RestDocsList], 0 -> AccRemoveSeqs; _ -> [OldSeq | AccRemoveSeqs] end, - merge_rev_trees(MergeConflicts, RestDocsList, RestOldInfo, - [NewInfo|AccNewInfos], RemoveSeqs, AccSeq+1) + merge_rev_trees(Limit, MergeConflicts, RestDocsList, RestOldInfo, + [NewInfo|AccNewInfos], RemoveSeqs, AccSeq+1) end. @@ -552,7 +584,8 @@ update_docs_int(Db, DocsList, NonRepDocs, MergeConflicts, FullCommit) -> #db{ fulldocinfo_by_id_btree = DocInfoByIdBTree, docinfo_by_seq_btree = DocInfoBySeqBTree, - update_seq = LastSeq + update_seq = LastSeq, + revs_limit = RevsLimit } = Db, Ids = [Id || [{_Client, #doc{id=Id}}|_] <- DocsList], % lookup up the old documents, if they exist. @@ -565,11 +598,9 @@ update_docs_int(Db, DocsList, NonRepDocs, MergeConflicts, FullCommit) -> end, Ids, OldDocLookups), % Merge the new docs into the revision trees. - {ok, NewDocInfos0, RemoveSeqs, NewSeq} = merge_rev_trees( + {ok, NewFullDocInfos, RemoveSeqs, NewSeq} = merge_rev_trees(RevsLimit, MergeConflicts, DocsList, OldDocInfos, [], [], LastSeq), - NewFullDocInfos = stem_full_doc_infos(Db, NewDocInfos0), - % All documents are now ready to write. {ok, Db2} = update_local_docs(Db, NonRepDocs), @@ -654,35 +685,35 @@ db_to_header(Db, Header) -> 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), - admins_ptr = Db#db.admins_ptr, + security_ptr = Db#db.security_ptr, revs_limit = Db#db.revs_limit}. -commit_data(#db{fd=Fd,header=OldHeader,fsync_options=FsyncOptions}=Db, Delay) -> - Header = db_to_header(Db, OldHeader), - if OldHeader == Header -> - Db; - Delay and (Db#db.waiting_delayed_commit == nil) -> - Db#db{waiting_delayed_commit= - erlang:send_after(1000, self(), delayed_commit)}; - Delay -> - Db; - true -> - if Db#db.waiting_delayed_commit /= nil -> - case erlang:cancel_timer(Db#db.waiting_delayed_commit) of - false -> receive delayed_commit -> ok after 0 -> ok end; - _ -> ok - end; - true -> ok - end, +commit_data(#db{waiting_delayed_commit=nil} = Db, true) -> + Db#db{waiting_delayed_commit=erlang:send_after(1000,self(),delayed_commit)}; +commit_data(Db, true) -> + Db; +commit_data(Db, _) -> + #db{ + fd = Fd, + filepath = Filepath, + header = OldHeader, + fsync_options = FsyncOptions, + waiting_delayed_commit = Timer + } = Db, + if is_reference(Timer) -> erlang:cancel_timer(Timer); true -> ok end, + case db_to_header(Db, OldHeader) of + OldHeader -> + Db#db{waiting_delayed_commit=nil}; + Header -> case lists:member(before_header, FsyncOptions) of - true -> ok = couch_file:sync(Fd); + true -> ok = couch_file:sync(Filepath); _ -> ok end, ok = couch_file:write_header(Fd, Header), case lists:member(after_header, FsyncOptions) of - true -> ok = couch_file:sync(Fd); + true -> ok = couch_file:sync(Filepath); _ -> ok end, @@ -696,21 +727,41 @@ copy_doc_attachments(#db{fd=SrcFd}=SrcDb, {Pos,_RevId}, SrcSp, DestFd) -> {ok, {BodyData, BinInfos}} = couch_db:read_doc(SrcDb, SrcSp), % copy the bin values NewBinInfos = lists:map( - fun({Name, {Type, BinSp, Len}}) when is_tuple(BinSp) orelse BinSp == null -> + fun({Name, {Type, BinSp, AttLen}}) when is_tuple(BinSp) orelse BinSp == null -> % 09 UPGRADE CODE - {NewBinSp, Len, Md5} = couch_stream:old_copy_to_new_stream(SrcFd, BinSp, Len, DestFd), - {Name, Type, NewBinSp, Len, Pos, Md5}; - ({Name, {Type, BinSp, Len}}) -> + {NewBinSp, AttLen, AttLen, Md5, _IdentityMd5} = + couch_stream:old_copy_to_new_stream(SrcFd, BinSp, AttLen, DestFd), + {Name, Type, NewBinSp, AttLen, AttLen, Pos, Md5, identity}; + ({Name, {Type, BinSp, AttLen}}) -> % 09 UPGRADE CODE - {NewBinSp, Len, Md5} = couch_stream:copy_to_new_stream(SrcFd, BinSp, DestFd), - {Name, Type, NewBinSp, Len, Pos, Md5}; - ({Name, Type, BinSp, Len, RevPos, <<>>}) when is_tuple(BinSp) orelse BinSp == null -> + {NewBinSp, AttLen, AttLen, Md5, _IdentityMd5} = + couch_stream:copy_to_new_stream(SrcFd, BinSp, DestFd), + {Name, Type, NewBinSp, AttLen, AttLen, Pos, Md5, identity}; + ({Name, Type, BinSp, AttLen, _RevPos, <<>>}) when + is_tuple(BinSp) orelse BinSp == null -> % 09 UPGRADE CODE - {NewBinSp, Len, Md5} = couch_stream:old_copy_to_new_stream(SrcFd, BinSp, Len, DestFd), - {Name, Type, NewBinSp, Len, Len, Md5}; - ({Name, Type, BinSp, Len, RevPos, Md5}) -> - {NewBinSp, Len, Md5} = couch_stream:copy_to_new_stream(SrcFd, BinSp, DestFd), - {Name, Type, NewBinSp, Len, RevPos, Md5} + {NewBinSp, AttLen, AttLen, Md5, _IdentityMd5} = + couch_stream:old_copy_to_new_stream(SrcFd, BinSp, AttLen, DestFd), + {Name, Type, NewBinSp, AttLen, AttLen, AttLen, Md5, identity}; + ({Name, Type, BinSp, AttLen, RevPos, Md5}) -> + % 010 UPGRADE CODE + {NewBinSp, AttLen, AttLen, Md5, _IdentityMd5} = + couch_stream:copy_to_new_stream(SrcFd, BinSp, DestFd), + {Name, Type, NewBinSp, AttLen, AttLen, RevPos, Md5, identity}; + ({Name, Type, BinSp, AttLen, DiskLen, RevPos, Md5, Enc1}) -> + {NewBinSp, AttLen, _, Md5, _IdentityMd5} = + couch_stream:copy_to_new_stream(SrcFd, BinSp, DestFd), + Enc = case Enc1 of + true -> + % 0110 UPGRADE CODE + gzip; + false -> + % 0110 UPGRADE CODE + identity; + _ -> + Enc1 + end, + {Name, Type, NewBinSp, AttLen, DiskLen, RevPos, Md5, Enc} end, BinInfos), {BodyData, NewBinInfos}. @@ -724,7 +775,10 @@ copy_rev_tree_attachments(SrcDb, DestFd, Tree) -> end, Tree). -copy_docs(Db, #db{fd=DestFd}=NewDb, InfoBySeq, Retry) -> +copy_docs(Db, #db{fd=DestFd}=NewDb, InfoBySeq0, Retry) -> + % COUCHDB-968, make sure we prune duplicates during compaction + InfoBySeq = lists:usort(fun(#doc_info{id=A}, #doc_info{id=B}) -> A =< B end, + InfoBySeq0), Ids = [Id || #doc_info{id=Id} <- InfoBySeq], LookupResults = couch_btree:lookup(Db#db.fulldocinfo_by_id_btree, Ids), @@ -790,7 +844,7 @@ copy_compact(Db, NewDb0, Retry) -> couch_task_status:set_update_frequency(500), {ok, _, {NewDb2, Uncopied, TotalChanges}} = - couch_btree:foldl(Db#db.docinfo_by_seq_btree, EnumBySeqFun, + couch_btree:foldl(Db#db.docinfo_by_seq_btree, EnumBySeqFun, {NewDb, [], 0}, [{start_key, NewDb#db.update_seq + 1}]), @@ -799,9 +853,9 @@ copy_compact(Db, NewDb0, Retry) -> 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}; + if NewDb3#db.security /= Db#db.security -> + {ok, Ptr} = couch_file:append_term(NewDb3#db.fd, Db#db.security), + NewDb4 = NewDb3#db{security=Db#db.security, security_ptr=Ptr}; true -> NewDb4 = NewDb3 end, @@ -815,7 +869,12 @@ start_copy_compact(#db{name=Name,filepath=Filepath}=Db) -> {ok, Fd} -> couch_task_status:add_task(<<"Database Compaction">>, <<Name/binary, " retry">>, <<"Starting">>), Retry = true, - {ok, Header} = couch_file:read_header(Fd); + case couch_file:read_header(Fd) of + {ok, Header} -> + ok; + no_valid_header -> + ok = couch_file:write_header(Fd, Header=#db_header{}) + end; {error, enoent} -> couch_task_status:add_task(<<"Database Compaction">>, Name, <<"Starting">>), {ok, Fd} = couch_file:open(CompactFile, [create]), @@ -823,8 +882,8 @@ start_copy_compact(#db{name=Name,filepath=Filepath}=Db) -> ok = couch_file:write_header(Fd, Header=#db_header{}) end, NewDb = init_db(Name, CompactFile, Fd, Header), + unlink(Fd), NewDb2 = copy_compact(Db, NewDb, Retry), - - gen_server:cast(Db#db.update_pid, {compact_done, CompactFile}), - close_db(NewDb2). + close_db(NewDb2), + gen_server:cast(Db#db.update_pid, {compact_done, CompactFile}). diff --git a/src/couchdb/couch_doc.erl b/src/couchdb/couch_doc.erl index ba5c7450..d15cd7de 100644 --- a/src/couchdb/couch_doc.erl +++ b/src/couchdb/couch_doc.erl @@ -13,7 +13,7 @@ -module(couch_doc). -export([to_doc_info/1,to_doc_info_path/1,parse_rev/1,parse_revs/1,rev_to_str/1,revs_to_strs/1]). --export([att_foldl/3,get_validate_doc_fun/1]). +-export([att_foldl/3,att_foldl_decode/3,get_validate_doc_fun/1]). -export([from_json_obj/1,to_json_obj/2,has_stubs/1, merge_stubs/2]). -export([validate_docid/1]). -export([doc_from_multi_part_stream/2]). @@ -27,8 +27,8 @@ to_json_rev(0, []) -> to_json_rev(Start, [FirstRevId|_]) -> [{<<"_rev">>, ?l2b([integer_to_list(Start),"-",revid_to_str(FirstRevId)])}]. -to_json_body(true, _Body) -> - [{<<"_deleted">>, true}]; +to_json_body(true, {Body}) -> + Body ++ [{<<"_deleted">>, true}]; to_json_body(false, {Body}) -> Body. @@ -73,35 +73,48 @@ to_json_meta(Meta) -> end, Meta). to_json_attachments(Attachments, Options) -> - case lists:member(attachments, Options) of - true -> % return all the binaries - to_json_attachments(Attachments, 0, lists:member(follows, Options)); - false -> - % note the default is [], because this sorts higher than all numbers. - % and will return all the binaries. - RevPos = proplists:get_value(atts_after_revpos, Options, []), - to_json_attachments(Attachments, RevPos, lists:member(follows, Options)) - end. - -to_json_attachments([], _RevPosIncludeAfter, _DataToFollow) -> + to_json_attachments( + Attachments, + lists:member(attachments, Options), + lists:member(follows, Options), + lists:member(att_encoding_info, Options) + ). + +to_json_attachments([], _OutputData, _DataToFollow, _ShowEncInfo) -> []; -to_json_attachments(Atts, RevPosIncludeAfter, DataToFollow) -> +to_json_attachments(Atts, OutputData, DataToFollow, ShowEncInfo) -> AttProps = lists:map( - fun(#att{len=Len}=Att) -> + fun(#att{disk_len=DiskLen, att_len=AttLen, encoding=Enc}=Att) -> {Att#att.name, {[ {<<"content_type">>, Att#att.type}, {<<"revpos">>, Att#att.revpos} ] ++ - if Att#att.revpos > RevPosIncludeAfter -> + if not OutputData orelse Att#att.data == stub -> + [{<<"length">>, DiskLen}, {<<"stub">>, true}]; + true -> if DataToFollow -> - [{<<"length">>, Len}, {<<"follows">>, true}]; + [{<<"length">>, DiskLen}, {<<"follows">>, true}]; true -> - [{<<"data">>, - couch_util:encodeBase64(att_to_iolist(Att))}] - end; - true -> - [{<<"length">>, Len}, {<<"stub">>, true}] - end + AttData = case Enc of + gzip -> + zlib:gunzip(att_to_bin(Att)); + identity -> + att_to_bin(Att) + end, + [{<<"data">>, base64:encode(AttData)}] + end + end ++ + case {ShowEncInfo, Enc} of + {false, _} -> + []; + {true, identity} -> + []; + {true, _} -> + [ + {<<"encoding">>, couch_util:to_binary(Enc)}, + {<<"encoded_length">>, AttLen} + ] + end }} end, Atts), [{<<"_attachments">>, {AttProps}}]. @@ -182,34 +195,38 @@ transfer_fields([{<<"_rev">>, _Rev} | Rest], Doc) -> transfer_fields([{<<"_attachments">>, {JsonBins}} | Rest], Doc) -> Atts = lists:map(fun({Name, {BinProps}}) -> - case proplists:get_value(<<"stub">>, BinProps) of + case couch_util:get_value(<<"stub">>, BinProps) of true -> - Type = proplists:get_value(<<"content_type">>, BinProps), - Length = proplists:get_value(<<"length">>, BinProps), - RevPos = proplists:get_value(<<"revpos">>, BinProps, 0), - #att{name=Name, data=stub, type=Type, len=Length, revpos=RevPos}; + Type = couch_util:get_value(<<"content_type">>, BinProps), + RevPos = couch_util:get_value(<<"revpos">>, BinProps, nil), + DiskLen = couch_util:get_value(<<"length">>, BinProps), + {Enc, EncLen} = att_encoding_info(BinProps), + #att{name=Name, data=stub, type=Type, att_len=EncLen, + disk_len=DiskLen, encoding=Enc, revpos=RevPos}; _ -> - Type = proplists:get_value(<<"content_type">>, BinProps, + Type = couch_util:get_value(<<"content_type">>, BinProps, ?DEFAULT_ATTACHMENT_CONTENT_TYPE), - RevPos = proplists:get_value(<<"revpos">>, BinProps, 0), - case proplists:get_value(<<"follows">>, BinProps) of + RevPos = couch_util:get_value(<<"revpos">>, BinProps, 0), + case couch_util:get_value(<<"follows">>, BinProps) of true -> - #att{name=Name, data=follows, type=Type, - len=proplists:get_value(<<"length">>, BinProps), - revpos=RevPos}; + DiskLen = couch_util:get_value(<<"length">>, BinProps), + {Enc, EncLen} = att_encoding_info(BinProps), + #att{name=Name, data=follows, type=Type, encoding=Enc, + att_len=EncLen, disk_len=DiskLen, revpos=RevPos}; _ -> - Value = proplists:get_value(<<"data">>, BinProps), - Bin = couch_util:decodeBase64(Value), - #att{name=Name, data=Bin, type=Type, len=size(Bin), - revpos=RevPos} + Value = couch_util:get_value(<<"data">>, BinProps), + Bin = base64:decode(Value), + LenBin = size(Bin), + #att{name=Name, data=Bin, type=Type, att_len=LenBin, + disk_len=LenBin, revpos=RevPos} end end end, JsonBins), transfer_fields(Rest, Doc#doc{atts=Atts}); transfer_fields([{<<"_revisions">>, {Props}} | Rest], Doc) -> - RevIds = proplists:get_value(<<"ids">>, Props), - Start = proplists:get_value(<<"start">>, Props), + RevIds = couch_util:get_value(<<"ids">>, Props), + Start = couch_util:get_value(<<"start">>, Props), if not is_integer(Start) -> throw({doc_validation, "_revisions.start isn't an integer."}); not is_list(RevIds) -> @@ -243,6 +260,16 @@ transfer_fields([{<<"_",Name/binary>>, _} | _], _) -> transfer_fields([Field | Rest], #doc{body=Fields}=Doc) -> transfer_fields(Rest, Doc#doc{body=[Field|Fields]}). +att_encoding_info(BinProps) -> + DiskLen = couch_util:get_value(<<"length">>, BinProps), + case couch_util:get_value(<<"encoding">>, BinProps) of + undefined -> + {identity, DiskLen}; + Enc -> + EncodedLen = couch_util:get_value(<<"encoded_length">>, BinProps, DiskLen), + {list_to_existing_atom(?b2l(Enc)), EncodedLen} + end. + to_doc_info(FullDocInfo) -> {DocInfo, _Path} = to_doc_info_path(FullDocInfo), DocInfo. @@ -272,33 +299,48 @@ to_doc_info_path(#full_doc_info{id=Id,rev_tree=Tree}) -> att_foldl(#att{data=Bin}, Fun, Acc) when is_binary(Bin) -> Fun(Bin, Acc); -att_foldl(#att{data={Fd,Sp},len=Len}, Fun, Acc) when is_tuple(Sp) orelse Sp == null -> +att_foldl(#att{data={Fd,Sp},att_len=Len}, Fun, Acc) when is_tuple(Sp) orelse Sp == null -> % 09 UPGRADE CODE couch_stream:old_foldl(Fd, Sp, Len, Fun, Acc); att_foldl(#att{data={Fd,Sp},md5=Md5}, Fun, Acc) -> couch_stream:foldl(Fd, Sp, Md5, Fun, Acc); -att_foldl(#att{data=DataFun,len=Len}, Fun, Acc) when is_function(DataFun) -> +att_foldl(#att{data=DataFun,att_len=Len}, Fun, Acc) when is_function(DataFun) -> fold_streamed_data(DataFun, Len, Fun, Acc). +att_foldl_decode(#att{data={Fd,Sp},md5=Md5,encoding=Enc}, Fun, Acc) -> + couch_stream:foldl_decode(Fd, Sp, Md5, Enc, Fun, Acc); +att_foldl_decode(#att{data=Fun2,att_len=Len, encoding=identity}, Fun, Acc) -> + fold_streamed_data(Fun2, Len, Fun, Acc). -att_to_iolist(#att{data=Bin}) when is_binary(Bin) -> +att_to_bin(#att{data=Bin}) when is_binary(Bin) -> Bin; -att_to_iolist(#att{data=Iolist}) when is_list(Iolist) -> - Iolist; -att_to_iolist(#att{data={Fd,Sp}}=Att) -> - lists:reverse(att_foldl(Att, - fun(Bin,Acc) -> [Bin|Acc] end, [])); -att_to_iolist(#att{data=DataFun, len=Len}) when is_function(DataFun)-> - lists:reverse(fold_streamed_data(DataFun, Len, - fun(Data, Acc) -> [Data | Acc] end, [])). +att_to_bin(#att{data=Iolist}) when is_list(Iolist) -> + iolist_to_binary(Iolist); +att_to_bin(#att{data={_Fd,_Sp}}=Att) -> + iolist_to_binary( + lists:reverse(att_foldl( + Att, + fun(Bin,Acc) -> [Bin|Acc] end, + [] + )) + ); +att_to_bin(#att{data=DataFun, att_len=Len}) when is_function(DataFun)-> + iolist_to_binary( + lists:reverse(fold_streamed_data( + DataFun, + Len, + fun(Data, Acc) -> [Data | Acc] end, + [] + )) + ). get_validate_doc_fun(#doc{body={Props}}=DDoc) -> - case proplists:get_value(<<"validate_doc_update">>, Props) of + case couch_util:get_value(<<"validate_doc_update">>, Props) of undefined -> nil; _Else -> - fun(EditDoc, DiskDoc, Ctx) -> - couch_query_servers:validate_doc_update(DDoc, EditDoc, DiskDoc, Ctx) + fun(EditDoc, DiskDoc, Ctx, SecObj) -> + couch_query_servers:validate_doc_update(DDoc, EditDoc, DiskDoc, Ctx, SecObj) end end. @@ -315,9 +357,10 @@ has_stubs([_Att|Rest]) -> merge_stubs(#doc{id=Id,atts=MemBins}=StubsDoc, #doc{atts=DiskBins}) -> BinDict = dict:from_list([{Name, Att} || #att{name=Name}=Att <- DiskBins]), MergedBins = lists:map( - fun(#att{name=Name, data=stub, revpos=RevPos}) -> + fun(#att{name=Name, data=stub, revpos=StubRevPos}) -> case dict:find(Name, BinDict) of - {ok, #att{revpos=RevPos}=DiskAtt} -> + {ok, #att{revpos=DiskRevPos}=DiskAtt} + when DiskRevPos == StubRevPos orelse StubRevPos == nil -> DiskAtt; _ -> throw({missing_stub, @@ -335,70 +378,102 @@ fold_streamed_data(RcvFun, LenLeft, Fun, Acc) when LenLeft > 0-> ResultAcc = Fun(Bin, Acc), fold_streamed_data(RcvFun, LenLeft - size(Bin), Fun, ResultAcc). -len_doc_to_multi_part_stream(Boundary,JsonBytes,Atts,AttsSinceRevPos) -> - 2 + % "--" - size(Boundary) + - 36 + % "\r\ncontent-type: application/json\r\n\r\n" - iolist_size(JsonBytes) + - 4 + % "\r\n--" - size(Boundary) + - + lists:foldl(fun(#att{revpos=RevPos,len=Len}, AccAttsSize) -> - if RevPos > AttsSinceRevPos -> - AccAttsSize + +len_doc_to_multi_part_stream(Boundary, JsonBytes, Atts, SendEncodedAtts) -> + AttsSize = lists:foldl(fun(#att{data=Data} = Att, AccAttsSize) -> + case Data of + stub -> + AccAttsSize; + _ -> + AccAttsSize + 4 + % "\r\n\r\n" - Len + + case SendEncodedAtts of + true -> + Att#att.att_len; + _ -> + Att#att.disk_len + end + 4 + % "\r\n--" - size(Boundary); - true -> - AccAttsSize + size(Boundary) end - end, 0, Atts) + - 2. % "--" - -doc_to_multi_part_stream(Boundary,JsonBytes,Atts,AttsSinceRevPos,WriteFun) -> - WriteFun([<<"--", Boundary/binary, - "\r\ncontent-type: application/json\r\n\r\n">>, - JsonBytes, <<"\r\n--", Boundary/binary>>]), - atts_to_mp(Atts, Boundary, WriteFun, AttsSinceRevPos). - -atts_to_mp([], _Boundary, WriteFun, _AttsSinceRevPos) -> + end, 0, Atts), + if AttsSize == 0 -> + {<<"application/json">>, iolist_size(JsonBytes)}; + true -> + {<<"multipart/related; boundary=\"", Boundary/binary, "\"">>, + 2 + % "--" + size(Boundary) + + 36 + % "\r\ncontent-type: application/json\r\n\r\n" + iolist_size(JsonBytes) + + 4 + % "\r\n--" + size(Boundary) + + + AttsSize + + 2 % "--" + } + end. + +doc_to_multi_part_stream(Boundary, JsonBytes, Atts, WriteFun, + SendEncodedAtts) -> + case lists:any(fun(#att{data=Data})-> Data /= stub end, Atts) of + true -> + WriteFun([<<"--", Boundary/binary, + "\r\ncontent-type: application/json\r\n\r\n">>, + JsonBytes, <<"\r\n--", Boundary/binary>>]), + atts_to_mp(Atts, Boundary, WriteFun, SendEncodedAtts); + false -> + WriteFun(JsonBytes) + end. + +atts_to_mp([], _Boundary, WriteFun, _SendEncAtts) -> WriteFun(<<"--">>); -atts_to_mp([#att{revpos=RevPos} = Att | RestAtts], Boundary, WriteFun, - AttsSinceRevPos) when RevPos > AttsSinceRevPos -> +atts_to_mp([#att{data=stub} | RestAtts], Boundary, WriteFun, + SendEncodedAtts) -> + atts_to_mp(RestAtts, Boundary, WriteFun, SendEncodedAtts); +atts_to_mp([Att | RestAtts], Boundary, WriteFun, + SendEncodedAtts) -> WriteFun(<<"\r\n\r\n">>), - att_foldl(Att, fun(Data, ok) -> WriteFun(Data) end, ok), + AttFun = case SendEncodedAtts of + false -> + fun att_foldl_decode/3; + true -> + fun att_foldl/3 + end, + AttFun(Att, fun(Data, _) -> WriteFun(Data) end, ok), WriteFun(<<"\r\n--", Boundary/binary>>), - atts_to_mp(RestAtts, Boundary, WriteFun, AttsSinceRevPos); -atts_to_mp([_ | RestAtts], Boundary, WriteFun, AttsSinceRevPos) -> - atts_to_mp(RestAtts, Boundary, WriteFun, AttsSinceRevPos). + atts_to_mp(RestAtts, Boundary, WriteFun, SendEncodedAtts). doc_from_multi_part_stream(ContentType, DataFun) -> - Parser = spawn_link(fun() -> + Self = self(), + Parser = spawn_link(fun() -> couch_httpd:parse_multipart_request(ContentType, DataFun, - fun(Next)-> mp_parse_doc(Next, []) end) + fun(Next)-> mp_parse_doc(Next, []) end), + unlink(Self) end), Parser ! {get_doc_bytes, self()}, - receive {doc_bytes, DocBytes} -> ok end, - Doc = from_json_obj(?JSON_DECODE(DocBytes)), - % go through the attachments looking for 'follows' in the data, - % replace with function that reads the data from MIME stream. - ReadAttachmentDataFun = fun() -> - Parser ! {get_bytes, self()}, - receive {bytes, Bytes} -> Bytes end - end, - Atts2 = lists:map( - fun(#att{data=follows}=A) -> - A#att{data=ReadAttachmentDataFun}; - (A) -> - A - end, Doc#doc.atts), - Doc#doc{atts=Atts2}. + receive + {doc_bytes, DocBytes} -> + Doc = from_json_obj(?JSON_DECODE(DocBytes)), + % go through the attachments looking for 'follows' in the data, + % replace with function that reads the data from MIME stream. + ReadAttachmentDataFun = fun() -> + Parser ! {get_bytes, self()}, + receive {bytes, Bytes} -> Bytes end + end, + Atts2 = lists:map( + fun(#att{data=follows}=A) -> + A#att{data=ReadAttachmentDataFun}; + (A) -> + A + end, Doc#doc.atts), + {ok, Doc#doc{atts=Atts2}} + end. mp_parse_doc({headers, H}, []) -> - {"application/json", _} = proplists:get_value("content-type", H), - fun (Next) -> - mp_parse_doc(Next, []) + case couch_util:get_value("content-type", H) of + {"application/json", _} -> + fun (Next) -> + mp_parse_doc(Next, []) + end end; mp_parse_doc({body, Bytes}, AccBytes) -> fun (Next) -> diff --git a/src/couchdb/couch_external_server.erl b/src/couchdb/couch_external_server.erl index 8e495320..045fcee9 100644 --- a/src/couchdb/couch_external_server.erl +++ b/src/couchdb/couch_external_server.erl @@ -50,9 +50,11 @@ terminate(_Reason, {_Name, _Command, Pid}) -> handle_call({execute, JsonReq}, _From, {Name, Command, Pid}) -> {reply, couch_os_process:prompt(Pid, JsonReq), {Name, Command, Pid}}. +handle_info({'EXIT', _Pid, normal}, State) -> + {noreply, State}; handle_info({'EXIT', Pid, Reason}, {Name, Command, Pid}) -> ?LOG_INFO("EXTERNAL: Process for ~s exiting. (reason: ~w)", [Name, Reason]), - {stop, normal, {Name, Command, Pid}}. + {stop, Reason, {Name, Command, Pid}}. handle_cast(stop, {Name, Command, Pid}) -> ?LOG_INFO("EXTERNAL: Shutting down ~s", [Name]), diff --git a/src/couchdb/couch_file.erl b/src/couchdb/couch_file.erl index 5904260c..fbfd6c6a 100644 --- a/src/couchdb/couch_file.erl +++ b/src/couchdb/couch_file.erl @@ -19,7 +19,8 @@ -record(file, { fd, - tail_append_begin=0 % 09 UPGRADE CODE + tail_append_begin = 0, % 09 UPGRADE CODE + eof = 0 }). -export([open/1, open/2, close/1, bytes/1, sync/1, append_binary/2,old_pread/3]). @@ -27,6 +28,7 @@ -export([pread_binary/2, read_header/1, truncate/2, upgrade_old_header/2]). -export([append_term_md5/2,append_binary_md5/2]). -export([init/1, terminate/2, handle_call/3, handle_cast/2, code_change/3, handle_info/2]). +-export([delete/2,delete/3,init_delete_dir/1]). %%---------------------------------------------------------------------- %% Args: Valid Options are [create] and [create,overwrite]. @@ -88,8 +90,8 @@ append_binary(Fd, Bin) -> append_binary_md5(Fd, Bin) -> Size = iolist_size(Bin), - gen_server:call(Fd, {append_bin, - [<<1:1/integer,Size:31/integer>>, erlang:md5(Bin), Bin]}, infinity). + gen_server:call(Fd, {append_bin, + [<<1:1/integer,Size:31/integer>>, couch_util:md5(Bin), Bin]}, infinity). %%---------------------------------------------------------------------- @@ -118,33 +120,18 @@ pread_binary(Fd, Pos) -> pread_iolist(Fd, Pos) -> - {ok, LenIolist, NextPos} = read_raw_iolist(Fd, Pos, 4), - case iolist_to_binary(LenIolist) of - <<1:1/integer,Len:31/integer>> -> - {ok, Md5List, ValPos} = read_raw_iolist(Fd, NextPos, 16), - Md5 = iolist_to_binary(Md5List), - {ok, IoList, _} = read_raw_iolist(Fd,ValPos,Len), - case erlang:md5(IoList) of - Md5 -> ok; - _ -> throw(file_corruption) - end, + case gen_server:call(Fd, {pread_iolist, Pos}, infinity) of + {ok, IoList, <<>>} -> {ok, IoList}; - <<0:1/integer,Len:31/integer>> -> - {ok, Iolist, _} = read_raw_iolist(Fd, NextPos, Len), - {ok, Iolist} - end. - - -read_raw_iolist(Fd, Pos, Len) -> - BlockOffset = Pos rem ?SIZE_BLOCK, - TotalBytes = calculate_total_read_len(BlockOffset, Len), - {ok, <<RawBin:TotalBytes/binary>>, HasPrefixes} = gen_server:call(Fd, {pread, Pos, TotalBytes}, infinity), - if HasPrefixes -> - {ok, remove_block_prefixes(BlockOffset, RawBin), Pos + TotalBytes}; - true -> - % 09 UPGRADE CODE - <<ReturnBin:Len/binary, _/binary>> = RawBin, - {ok, [ReturnBin], Pos + Len} + {ok, IoList, Md5} -> + case couch_util:md5(IoList) of + Md5 -> + {ok, IoList}; + _ -> + exit({file_corruption, <<"file corruption">>}) + end; + Error -> + Error end. %%---------------------------------------------------------------------- @@ -172,17 +159,59 @@ truncate(Fd, Pos) -> %% or {error, Reason}. %%---------------------------------------------------------------------- +sync(Filepath) when is_list(Filepath) -> + {ok, Fd} = file:open(Filepath, [append, raw]), + try file:sync(Fd) after file:close(Fd) end; sync(Fd) -> gen_server:call(Fd, sync, infinity). %%---------------------------------------------------------------------- -%% Purpose: Close the file. Is performed asynchronously. +%% Purpose: Close the file. %% Returns: ok %%---------------------------------------------------------------------- close(Fd) -> - Result = gen_server:cast(Fd, close), - catch unlink(Fd), - Result. + MRef = erlang:monitor(process, Fd), + try + catch unlink(Fd), + catch exit(Fd, shutdown), + receive + {'DOWN', MRef, _, _, _} -> + ok + end + after + erlang:demonitor(MRef, [flush]) + end. + + +delete(RootDir, Filepath) -> + delete(RootDir, Filepath, true). + + +delete(RootDir, Filepath, Async) -> + DelFile = filename:join([RootDir,".delete", ?b2l(couch_uuids:random())]), + case file:rename(Filepath, DelFile) of + ok -> + if (Async) -> + spawn(file, delete, [DelFile]), + ok; + true -> + file:delete(DelFile) + end; + Error -> + Error + end. + + +init_delete_dir(RootDir) -> + Dir = filename:join(RootDir,".delete"), + % note: ensure_dir requires an actual filename companent, which is the + % reason for "foo". + filelib:ensure_dir(filename:join(Dir,"foo")), + filelib:fold_files(Dir, ".*", true, + fun(Filename, _) -> + ok = file:delete(Filename) + end, ok). + % 09 UPGRADE CODE old_pread(Fd, Pos, Len) -> @@ -204,7 +233,7 @@ read_header(Fd) -> write_header(Fd, Data) -> Bin = term_to_binary(Data), - Md5 = erlang:md5(Bin), + Md5 = couch_util:md5(Bin), % now we assemble the final header binary and write to disk FinalBin = <<Md5/binary, Bin/binary>>, gen_server:call(Fd, {write_header, FinalBin}, infinity). @@ -219,10 +248,11 @@ init_status_error(ReturnPid, Ref, Error) -> % server functions init({Filepath, Options, ReturnPid, Ref}) -> + process_flag(trap_exit, true), case lists:member(create, Options) of true -> filelib:ensure_dir(Filepath), - case file:open(Filepath, [read, write, raw, binary]) of + case file:open(Filepath, [read, append, raw, binary]) of {ok, Fd} -> {ok, Length} = file:position(Fd, eof), case Length > 0 of @@ -235,16 +265,14 @@ init({Filepath, Options, ReturnPid, Ref}) -> {ok, 0} = file:position(Fd, 0), ok = file:truncate(Fd), ok = file:sync(Fd), - couch_stats_collector:track_process_count( - {couchdb, open_os_files}), + maybe_track_open_os_files(Options), {ok, #file{fd=Fd}}; false -> ok = file:close(Fd), init_status_error(ReturnPid, Ref, file_exists) end; false -> - couch_stats_collector:track_process_count( - {couchdb, open_os_files}), + maybe_track_open_os_files(Options), {ok, #file{fd=Fd}} end; Error -> @@ -254,41 +282,63 @@ init({Filepath, Options, ReturnPid, Ref}) -> % open in read mode first, so we don't create the file if it doesn't exist. case file:open(Filepath, [read, raw]) of {ok, Fd_Read} -> - {ok, Fd} = file:open(Filepath, [read, write, raw, binary]), + {ok, Fd} = file:open(Filepath, [read, append, raw, binary]), ok = file:close(Fd_Read), - couch_stats_collector:track_process_count({couchdb, open_os_files}), - {ok, #file{fd=Fd}}; + maybe_track_open_os_files(Options), + {ok, Length} = file:position(Fd, eof), + {ok, #file{fd=Fd, eof=Length}}; Error -> init_status_error(ReturnPid, Ref, Error) end end. +maybe_track_open_os_files(FileOptions) -> + case lists:member(sys_db, FileOptions) of + true -> + ok; + false -> + couch_stats_collector:track_process_count({couchdb, open_os_files}) + end. terminate(_Reason, _Fd) -> ok. +handle_call({pread_iolist, Pos}, _From, File) -> + {LenIolist, NextPos} = read_raw_iolist_int(File, Pos, 4), + case iolist_to_binary(LenIolist) of + <<1:1/integer,Len:31/integer>> -> % an MD5-prefixed term + {Md5AndIoList, _} = read_raw_iolist_int(File, NextPos, Len+16), + {Md5, IoList} = extract_md5(Md5AndIoList), + {reply, {ok, IoList, Md5}, File}; + <<0:1/integer,Len:31/integer>> -> + {Iolist, _} = read_raw_iolist_int(File, NextPos, Len), + {reply, {ok, Iolist, <<>>}, File} + end; handle_call({pread, Pos, Bytes}, _From, #file{fd=Fd,tail_append_begin=TailAppendBegin}=File) -> {ok, Bin} = file:pread(Fd, Pos, Bytes), {reply, {ok, Bin, Pos >= TailAppendBegin}, File}; -handle_call(bytes, _From, #file{fd=Fd}=File) -> - {reply, file:position(Fd, eof), File}; +handle_call(bytes, _From, #file{eof=Length}=File) -> + {reply, {ok, Length}, File}; handle_call(sync, _From, #file{fd=Fd}=File) -> {reply, file:sync(Fd), File}; handle_call({truncate, Pos}, _From, #file{fd=Fd}=File) -> {ok, Pos} = file:position(Fd, Pos), - {reply, file:truncate(Fd), File}; -handle_call({append_bin, Bin}, _From, #file{fd=Fd}=File) -> - {ok, Pos} = file:position(Fd, eof), + case file:truncate(Fd) of + ok -> + {reply, ok, File#file{eof=Pos}}; + Error -> + {reply, Error, File} + end; +handle_call({append_bin, Bin}, _From, #file{fd=Fd, eof=Pos}=File) -> Blocks = make_blocks(Pos rem ?SIZE_BLOCK, Bin), - case file:pwrite(Fd, Pos, Blocks) of + case file:write(Fd, Blocks) of ok -> - {reply, {ok, Pos}, File}; + {reply, {ok, Pos}, File#file{eof=Pos+iolist_size(Blocks)}}; Error -> {reply, Error, File} end; -handle_call({write_header, Bin}, _From, #file{fd=Fd}=File) -> - {ok, Pos} = file:position(Fd, eof), +handle_call({write_header, Bin}, _From, #file{fd=Fd, eof=Pos}=File) -> BinSize = size(Bin), case Pos rem ?SIZE_BLOCK of 0 -> @@ -296,16 +346,21 @@ handle_call({write_header, Bin}, _From, #file{fd=Fd}=File) -> BlockOffset -> Padding = <<0:(8*(?SIZE_BLOCK-BlockOffset))>> end, - FinalBin = [Padding, <<1, BinSize:32/integer>> | make_blocks(1, [Bin])], - {reply, file:pwrite(Fd, Pos, FinalBin), File}; + FinalBin = [Padding, <<1, BinSize:32/integer>> | make_blocks(5, [Bin])], + case file:write(Fd, FinalBin) of + ok -> + {reply, ok, File#file{eof=Pos+iolist_size(FinalBin)}}; + Error -> + {reply, Error, File} + end; handle_call({upgrade_old_header, Prefix}, _From, #file{fd=Fd}=File) -> case (catch read_old_header(Fd, Prefix)) of {ok, Header} -> - {ok, TailAppendBegin} = file:position(Fd, eof), + TailAppendBegin = File#file.eof, Bin = term_to_binary(Header), - Md5 = erlang:md5(Bin), + Md5 = couch_util:md5(Bin), % now we assemble the final header binary and write to disk FinalBin = <<Md5/binary, Bin/binary>>, {reply, ok, _} = handle_call({write_header, FinalBin}, ok, File), @@ -321,8 +376,7 @@ handle_call({upgrade_old_header, Prefix}, _From, #file{fd=Fd}=File) -> end; -handle_call(find_header, _From, #file{fd=Fd}=File) -> - {ok, Pos} = file:position(Fd, eof), +handle_call(find_header, _From, #file{fd=Fd, eof=Pos}=File) -> {reply, find_header(Fd, Pos div ?SIZE_BLOCK), File}. % 09 UPGRADE CODE @@ -386,7 +440,7 @@ extract_header(Prefix, Bin) -> case HeaderPrefix of Prefix -> % check the integrity signature - case erlang:md5(TermBin) == Sig of + case couch_util:md5(TermBin) == Sig of true -> Header = binary_to_term(TermBin), {ok, Header}; @@ -416,7 +470,7 @@ write_old_header(Fd, Prefix, Data) -> ok = file:sync(Fd), % pad out the header with zeros, then take the md5 hash PadZeros = <<0:(8*(?HEADER_SIZE - FilledSize2))>>, - Sig = erlang:md5([TermBin2, PadZeros]), + Sig = couch_util:md5([TermBin2, PadZeros]), % now we assemble the final header binary and write to disk WriteBin = <<Prefix/binary, TermBin2/binary, PadZeros/binary, Sig/binary>>, ?HEADER_SIZE = size(WriteBin), % sanity check @@ -432,6 +486,8 @@ handle_cast(close, Fd) -> code_change(_OldVsn, State, _Extra) -> {ok, State}. +handle_info({'EXIT', _, normal}, Fd) -> + {noreply, Fd}; handle_info({'EXIT', _, Reason}, Fd) -> {stop, Reason, Fd}. @@ -454,9 +510,28 @@ load_header(Fd, Block) -> file:pread(Fd, (Block*?SIZE_BLOCK) + 5, TotalBytes), <<Md5Sig:16/binary, HeaderBin/binary>> = iolist_to_binary(remove_block_prefixes(1, RawBin)), - Md5Sig = erlang:md5(HeaderBin), + Md5Sig = couch_util:md5(HeaderBin), {ok, HeaderBin}. +-spec read_raw_iolist_int(#file{}, Pos::non_neg_integer(), Len::non_neg_integer()) -> + {Data::iolist(), CurPos::non_neg_integer()}. +read_raw_iolist_int(#file{fd=Fd, tail_append_begin=TAB}, Pos, Len) -> + BlockOffset = Pos rem ?SIZE_BLOCK, + TotalBytes = calculate_total_read_len(BlockOffset, Len), + {ok, <<RawBin:TotalBytes/binary>>} = file:pread(Fd, Pos, TotalBytes), + if Pos >= TAB -> + {remove_block_prefixes(BlockOffset, RawBin), Pos + TotalBytes}; + true -> + % 09 UPGRADE CODE + <<ReturnBin:Len/binary, _/binary>> = RawBin, + {[ReturnBin], Pos + Len} + end. + +-spec extract_md5(iolist()) -> {binary(), iolist()}. +extract_md5(FullIoList) -> + {Md5List, IoList} = split_iolist(FullIoList, 16, []), + {iolist_to_binary(Md5List), IoList}. + calculate_total_read_len(0, FinalLen) -> calculate_total_read_len(1, FinalLen) + 1; calculate_total_read_len(BlockOffset, FinalLen) -> @@ -495,6 +570,11 @@ make_blocks(BlockOffset, IoList) -> IoList end. +%% @doc Returns a tuple where the first element contains the leading SplitAt +%% bytes of the original iolist, and the 2nd element is the tail. If SplitAt +%% is larger than byte_size(IoList), return the difference. +-spec split_iolist(IoList::iolist(), SplitAt::non_neg_integer(), Acc::list()) -> + {iolist(), iolist()} | non_neg_integer(). split_iolist(List, 0, BeginAcc) -> {lists:reverse(BeginAcc), List}; split_iolist([], SplitAt, _BeginAcc) -> diff --git a/src/couchdb/couch_httpd.erl b/src/couchdb/couch_httpd.erl index 252ecdb7..d24822aa 100644 --- a/src/couchdb/couch_httpd.erl +++ b/src/couchdb/couch_httpd.erl @@ -13,11 +13,11 @@ -module(couch_httpd). -include("couch_db.hrl"). --export([start_link/0, stop/0, handle_request/5]). +-export([start_link/0, stop/0, handle_request/7]). -export([header_value/2,header_value/3,qs_value/2,qs_value/3,qs/1,path/1,absolute_uri/2,body_length/1]). -export([verify_is_server_admin/1,unquote/1,quote/1,recv/2,recv_chunked/4,error_info/1]). --export([make_fun_spec_strs/1]). +-export([make_fun_spec_strs/1, make_arity_1_fun/1]). -export([parse_form/1,json_body/1,json_body_obj/1,body/1,doc_etag/1, make_etag/1, etag_respond/3]). -export([primary_header_value/2,partition/1,serve_file/3,serve_file/4, server_header/0]). -export([start_chunked_response/3,send_chunk/2,log_request/2]). @@ -25,6 +25,7 @@ -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, send_redirect/2,send_chunked_error/2]). -export([send_json/2,send_json/3,send_json/4,last_chunk/1,parse_multipart_request/3]). +-export([accepted_encodings/1,handle_request_int/5,validate_referer/1,validate_ctype/2]). start_link() -> % read config and register for configuration changes @@ -34,7 +35,13 @@ start_link() -> BindAddress = couch_config:get("httpd", "bind_address", any), Port = couch_config:get("httpd", "port", "5984"), - + MaxConnections = couch_config:get("httpd", "max_connections", "2048"), + VirtualHosts = couch_config:get("vhosts"), + VhostGlobals = re:split( + couch_config:get("httpd", "vhost_global_handlers", ""), + ", ?", + [{return, list}] + ), DefaultSpec = "{couch_httpd_db, handle_request}", DefaultFun = make_arity_1_fun( couch_config:get("httpd", "default_handler", DefaultSpec) @@ -60,7 +67,8 @@ start_link() -> DesignUrlHandlers = dict:from_list(DesignUrlHandlersList), Loop = fun(Req)-> apply(?MODULE, handle_request, [ - Req, DefaultFun, UrlHandlers, DbUrlHandlers, DesignUrlHandlers + Req, DefaultFun, UrlHandlers, DbUrlHandlers, DesignUrlHandlers, + VirtualHosts, VhostGlobals ]) end, @@ -70,7 +78,8 @@ start_link() -> {loop, Loop}, {name, ?MODULE}, {ip, BindAddress}, - {port, Port} + {port, Port}, + {max, MaxConnections} ]) of {ok, MochiPid} -> {ok, MochiPid}; {error, Reason} -> @@ -83,11 +92,15 @@ start_link() -> ?MODULE:stop(); ("httpd", "port") -> ?MODULE:stop(); + ("httpd", "max_connections") -> + ?MODULE:stop(); ("httpd", "default_handler") -> ?MODULE:stop(); ("httpd_global_handlers", _) -> ?MODULE:stop(); ("httpd_db_handlers", _) -> + ?MODULE:stop(); + ("vhosts", _) -> ?MODULE:stop() end, Pid), @@ -121,14 +134,58 @@ make_arity_3_fun(SpecStr) -> % SpecStr is "{my_module, my_fun}, {my_module2, my_fun2}" make_fun_spec_strs(SpecStr) -> - [FunSpecStr || FunSpecStr <- re:split(SpecStr, "(?<=})\\s*,\\s*(?={)", [{return, list}])]. + re:split(SpecStr, "(?<=})\\s*,\\s*(?={)", [{return, list}]). stop() -> mochiweb_http:stop(?MODULE). +%% + +% if there's a vhost definition that matches the request, redirect internally +redirect_to_vhost(MochiReq, DefaultFun, + UrlHandlers, DbUrlHandlers, DesignUrlHandlers, VhostTarget) -> + + Path = MochiReq:get(raw_path), + Target = VhostTarget ++ Path, + ?LOG_DEBUG("Vhost Target: '~p'~n", [Target]), + % build a new mochiweb request + MochiReq1 = mochiweb_request:new(MochiReq:get(socket), + MochiReq:get(method), + Target, + MochiReq:get(version), + MochiReq:get(headers)), + % cleanup, It force mochiweb to reparse raw uri. + MochiReq1:cleanup(), + + handle_request_int(MochiReq1, DefaultFun, + UrlHandlers, DbUrlHandlers, DesignUrlHandlers). handle_request(MochiReq, DefaultFun, - UrlHandlers, DbUrlHandlers, DesignUrlHandlers) -> + UrlHandlers, DbUrlHandlers, DesignUrlHandlers, VirtualHosts, VhostGlobals) -> + + % grab Host from Req + Vhost = MochiReq:get_header_value("Host"), + + % find Vhost in config + case couch_util:get_value(Vhost, VirtualHosts) of + undefined -> % business as usual + handle_request_int(MochiReq, DefaultFun, + UrlHandlers, DbUrlHandlers, DesignUrlHandlers); + VhostTarget -> + case vhost_global(VhostGlobals, MochiReq) of + true ->% global handler for vhosts + handle_request_int(MochiReq, DefaultFun, + UrlHandlers, DbUrlHandlers, DesignUrlHandlers); + _Else -> + % do rewrite + redirect_to_vhost(MochiReq, DefaultFun, + UrlHandlers, DbUrlHandlers, DesignUrlHandlers, VhostTarget) + end + end. + + +handle_request_int(MochiReq, DefaultFun, + UrlHandlers, DbUrlHandlers, DesignUrlHandlers) -> Begin = now(), AuthenticationSrcs = make_fun_spec_strs( couch_config:get("httpd", "authentication_handlers")), @@ -161,9 +218,26 @@ handle_request(MochiReq, DefaultFun, Meth -> couch_util:to_existing_atom(Meth) end, increment_method_stats(Method1), + + % allow broken HTTP clients to fake a full method vocabulary with an X-HTTP-METHOD-OVERRIDE header + MethodOverride = MochiReq:get_primary_header_value("X-HTTP-Method-Override"), + Method2 = case lists:member(MethodOverride, ["GET", "HEAD", "POST", "PUT", "DELETE", "TRACE", "CONNECT", "COPY"]) of + true -> + ?LOG_INFO("MethodOverride: ~s (real method was ~s)", [MethodOverride, Method1]), + case Method1 of + 'POST' -> couch_util:to_existing_atom(MethodOverride); + _ -> + % Ignore X-HTTP-Method-Override when the original verb isn't POST. + % I'd like to send a 406 error to the client, but that'd require a nasty refactor. + % throw({not_acceptable, <<"X-HTTP-Method-Override may only be used with POST requests.">>}) + Method1 + end; + _ -> Method1 + end, + % alias HEAD to GET as mochiweb takes care of stripping the body - Method = case Method1 of - 'HEAD' -> 'GET'; + Method = case Method2 of + 'HEAD' -> 'GET'; Other -> Other end, @@ -174,7 +248,9 @@ handle_request(MochiReq, DefaultFun, path_parts = [list_to_binary(couch_httpd:unquote(Part)) || Part <- string:tokens(Path, "/")], db_url_handlers = DbUrlHandlers, - design_url_handlers = DesignUrlHandlers + design_url_handlers = DesignUrlHandlers, + default_fun = DefaultFun, + url_handlers = UrlHandlers }, HandlerFun = couch_util:dict_find(HandlerKey, UrlHandlers, DefaultFun), @@ -191,8 +267,15 @@ handle_request(MochiReq, DefaultFun, throw:{http_head_abort, Resp0} -> {ok, Resp0}; throw:{invalid_json, S} -> - ?LOG_ERROR("attempted upload of invalid JSON ~s", [S]), + ?LOG_ERROR("attempted upload of invalid JSON (set log_level to debug to log it)", []), + ?LOG_DEBUG("Invalid JSON: ~p",[S]), send_error(HttpReq, {bad_request, "invalid UTF-8 JSON"}); + throw:unacceptable_encoding -> + ?LOG_ERROR("unsupported encoding method for the response", []), + send_error(HttpReq, {not_acceptable, "unsupported encoding"}); + throw:bad_accept_encoding_value -> + ?LOG_ERROR("received invalid Accept-Encoding header", []), + send_error(HttpReq, bad_request); exit:normal -> exit(normal); throw:Error -> @@ -243,6 +326,45 @@ authenticate_request(Response, _AuthSrcs) -> increment_method_stats(Method) -> couch_stats_collector:increment({httpd_request_methods, Method}). +% if so, then it will not be rewritten, but will run as a normal couchdb request. +% normally you'd use this for _uuids _utils and a few of the others you want to keep available on vhosts. You can also use it to make databases 'global'. +vhost_global(VhostGlobals, MochiReq) -> + "/" ++ Path = MochiReq:get(path), + Front = case partition(Path) of + {"", "", ""} -> + "/"; % Special case the root url handler + {FirstPart, _, _} -> + FirstPart + end, + [true] == [true||V <- VhostGlobals, V == Front]. + +validate_referer(Req) -> + Host = host_for_request(Req), + Referer = header_value(Req, "Referer", fail), + case Referer of + fail -> + throw({bad_request, <<"Referer header required.">>}); + Referer -> + {_,RefererHost,_,_,_} = mochiweb_util:urlsplit(Referer), + if + RefererHost =:= Host -> ok; + true -> throw({bad_request, <<"Referer header must match host.">>}) + end + end. + +validate_ctype(Req, Ctype) -> + case couch_httpd:header_value(Req, "Content-Type") of + undefined -> + throw({bad_ctype, "Content-Type must be "++Ctype}); + ReqCtype -> + % ?LOG_ERROR("Ctype ~p ReqCtype ~p",[Ctype,ReqCtype]), + case re:split(ReqCtype, ";", [{return, list}]) of + [Ctype] -> ok; + [Ctype, _Rest] -> ok; + _Else -> + throw({bad_ctype, "Content-Type must be "++Ctype}) + end + end. % Utilities @@ -261,7 +383,17 @@ header_value(#httpd{mochi_req=MochiReq}, Key, Default) -> primary_header_value(#httpd{mochi_req=MochiReq}, Key) -> MochiReq:get_primary_header_value(Key). -serve_file(#httpd{mochi_req=MochiReq}=Req, RelativePath, DocumentRoot) -> +accepted_encodings(#httpd{mochi_req=MochiReq}) -> + case MochiReq:accepted_encodings(["gzip", "identity"]) of + bad_accept_encoding_value -> + throw(bad_accept_encoding_value); + [] -> + throw(unacceptable_encoding); + EncList -> + EncList + end. + +serve_file(Req, RelativePath, DocumentRoot) -> serve_file(Req, RelativePath, DocumentRoot, []). serve_file(#httpd{mochi_req=MochiReq}=Req, RelativePath, DocumentRoot, ExtraHeaders) -> @@ -272,7 +404,7 @@ qs_value(Req, Key) -> qs_value(Req, Key, undefined). qs_value(Req, Key, Default) -> - proplists:get_value(Key, qs(Req), Default). + couch_util:get_value(Key, qs(Req), Default). qs(#httpd{mochi_req=MochiReq}) -> MochiReq:parse_qs(). @@ -280,19 +412,22 @@ qs(#httpd{mochi_req=MochiReq}) -> path(#httpd{mochi_req=MochiReq}) -> MochiReq:get(path). -absolute_uri(#httpd{mochi_req=MochiReq}, Path) -> +host_for_request(#httpd{mochi_req=MochiReq}) -> XHost = couch_config:get("httpd", "x_forwarded_host", "X-Forwarded-Host"), - Host = case MochiReq:get_header_value(XHost) of + case MochiReq:get_header_value(XHost) of undefined -> case MochiReq:get_header_value("Host") of - undefined -> + undefined -> {ok, {Address, Port}} = inet:sockname(MochiReq:get(socket)), inet_parse:ntoa(Address) ++ ":" ++ integer_to_list(Port); Value1 -> Value1 end; Value -> Value - end, + end. + +absolute_uri(#httpd{mochi_req=MochiReq}=Req, Path) -> + Host = host_for_request(Req), XSsl = couch_config:get("httpd", "x_forwarded_ssl", "X-Forwarded-Ssl"), Scheme = case MochiReq:get_header_value(XSsl) of "on" -> "https"; @@ -362,7 +497,7 @@ doc_etag(#doc{revs={Start, [DiskRev|_]}}) -> "\"" ++ ?b2l(couch_doc:rev_to_str({Start, DiskRev})) ++ "\"". make_etag(Term) -> - <<SigInt:128/integer>> = erlang:md5(term_to_binary(Term)), + <<SigInt:128/integer>> = couch_util:md5(term_to_binary(Term)), list_to_binary("\"" ++ lists:flatten(io_lib:format("~.36B",[SigInt])) ++ "\""). etag_match(Req, CurrentEtag) when is_binary(CurrentEtag) -> @@ -391,10 +526,10 @@ verify_is_server_admin(#user_ctx{roles=Roles}) -> false -> throw({unauthorized, <<"You are not a server admin.">>}) end. -log_request(#httpd{mochi_req=MochiReq,peer=Peer,method=Method}, Code) -> +log_request(#httpd{mochi_req=MochiReq,peer=Peer}, Code) -> ?LOG_INFO("~s - - ~p ~s ~B", [ Peer, - Method, + couch_util:to_existing_atom(MochiReq:get(method)), MochiReq:get(raw_path), couch_util:to_integer(Code) ]). @@ -513,8 +648,18 @@ start_jsonp(Req) -> [] -> []; CallBack -> try - validate_callback(CallBack), - CallBack ++ "(" + % make sure jsonp is configured on (default off) + case couch_config:get("httpd", "allow_jsonp", "false") of + "true" -> + validate_callback(CallBack), + CallBack ++ "("; + _Else -> + % this could throw an error message, but instead we just ignore the + % jsonp parameter + % throw({bad_request, <<"JSONP must be configured before using.">>}) + put(jsonp, no_jsonp), + [] + end catch Error -> put(jsonp, no_jsonp), @@ -581,7 +726,7 @@ error_info({bad_ctype, Reason}) -> error_info({error, illegal_database_name}) -> {400, <<"illegal_database_name">>, <<"Only lowercase characters (a-z), " "digits (0-9), and any of the characters _, $, (, ), +, -, and / " - "are allowed">>}; + "are allowed. Must begin with a letter.">>}; error_info({missing_stub, Reason}) -> {412, <<"missing_stub">>, Reason}; error_info({Error, Reason}) -> @@ -589,28 +734,73 @@ error_info({Error, Reason}) -> error_info(Error) -> {500, <<"unknown_error">>, couch_util:to_binary(Error)}. -send_error(_Req, {already_sent, Resp, _Error}) -> - {ok, Resp}; - -send_error(#httpd{mochi_req=MochiReq}=Req, Error) -> - {Code, ErrorStr, ReasonStr} = error_info(Error), - Headers = if Code == 401 -> +error_headers(#httpd{mochi_req=MochiReq}=Req, Code, ErrorStr, ReasonStr) -> + if Code == 401 -> % this is where the basic auth popup is triggered case MochiReq:get_header_value("X-CouchDB-WWW-Authenticate") of undefined -> case couch_config:get("httpd", "WWW-Authenticate", nil) of nil -> - []; + % If the client is a browser and the basic auth popup isn't turned on + % redirect to the session page. + case ErrorStr of + <<"unauthorized">> -> + case couch_config:get("couch_httpd_auth", "authentication_redirect", nil) of + nil -> {Code, []}; + AuthRedirect -> + case couch_config:get("couch_httpd_auth", "require_valid_user", "false") of + "true" -> + % send the browser popup header no matter what if we are require_valid_user + {Code, [{"WWW-Authenticate", "Basic realm=\"server\""}]}; + _False -> + % if the accept header matches html, then do the redirect. else proceed as usual. + Accepts = case MochiReq:get_header_value("Accept") of + undefined -> + % According to the HTTP 1.1 spec, if the Accept + % header is missing, it means the client accepts + % all media types. + "html"; + Else -> + Else + end, + case re:run(Accepts, "\\bhtml\\b", + [{capture, none}, caseless]) of + nomatch -> + {Code, []}; + match -> + AuthRedirectBin = ?l2b(AuthRedirect), + % Redirect to the path the user requested, not + % the one that is used internally. + UrlReturnRaw = case MochiReq:get_header_value("x-couchdb-vhost-path") of + undefined -> MochiReq:get(path); + VHostPath -> VHostPath + end, + UrlReturn = ?l2b(couch_util:url_encode(UrlReturnRaw)), + UrlReason = ?l2b(couch_util:url_encode(ReasonStr)), + {302, [{"Location", couch_httpd:absolute_uri(Req, <<AuthRedirectBin/binary,"?return=",UrlReturn/binary,"&reason=",UrlReason/binary>>)}]} + end + end + end; + _Else -> + {Code, []} + end; Type -> - [{"WWW-Authenticate", Type}] + {Code, [{"WWW-Authenticate", Type}]} end; Type -> - [{"WWW-Authenticate", Type}] + {Code, [{"WWW-Authenticate", Type}]} end; true -> - [] - end, - send_error(Req, Code, Headers, ErrorStr, ReasonStr). + {Code, []} + end. + +send_error(_Req, {already_sent, Resp, _Error}) -> + {ok, Resp}; + +send_error(Req, Error) -> + {Code, ErrorStr, ReasonStr} = error_info(Error), + {Code1, Headers} = error_headers(Req, Code, ErrorStr, ReasonStr), + send_error(Req, Code1, Headers, ErrorStr, ReasonStr). send_error(Req, Code, ErrorStr, ReasonStr) -> send_error(Req, Code, [], ErrorStr, ReasonStr). @@ -667,22 +857,24 @@ parse_multipart_request(ContentType, DataFun, Callback) -> buffer= <<>>, data_fun=DataFun, callback=Callback}, - {Mp2, _NilCallback} = read_until(Mp, <<"--", Boundary0/binary>>, + {Mp2, _NilCallback} = read_until(Mp, <<"--", Boundary0/binary>>, fun(Next)-> nil_callback(Next) end), - #mp{buffer=Buffer, data_fun=DataFun2, callback=Callback2} = + #mp{buffer=Buffer, data_fun=DataFun2, callback=Callback2} = parse_part_header(Mp2), {Buffer, DataFun2, Callback2}. nil_callback(_Data)-> fun(Next) -> nil_callback(Next) end. -get_boundary(ContentType) -> - {"multipart/" ++ _, Opts} = mochiweb_util:parse_header(ContentType), - case proplists:get_value("boundary", Opts) of +get_boundary({"multipart/" ++ _, Opts}) -> + case couch_util:get_value("boundary", Opts) of S when is_list(S) -> S - end. - + end; +get_boundary(ContentType) -> + {"multipart/" ++ _ , Opts} = mochiweb_util:parse_header(ContentType), + get_boundary({"multipart/", Opts}). + split_header(<<>>) -> @@ -700,6 +892,11 @@ read_until(#mp{data_fun=DataFun, buffer=Buffer}=Mp, Pattern, Callback) -> {Buffer2, DataFun2} = DataFun(), Buffer3 = iolist_to_binary(Buffer2), read_until(Mp#mp{data_fun=DataFun2,buffer=Buffer3}, Pattern, Callback2); + {partial, 0} -> + {NewData, DataFun2} = DataFun(), + read_until(Mp#mp{data_fun=DataFun2, + buffer= iolist_to_binary([Buffer,NewData])}, + Pattern, Callback); {partial, Skip} -> <<DataChunk:Skip/binary, Rest/binary>> = Buffer, Callback2 = Callback(DataChunk), @@ -707,6 +904,10 @@ read_until(#mp{data_fun=DataFun, buffer=Buffer}=Mp, Pattern, Callback) -> read_until(Mp#mp{data_fun=DataFun2, buffer= iolist_to_binary([Rest | NewData])}, Pattern, Callback2); + {exact, 0} -> + PatternLen = size(Pattern), + <<_:PatternLen/binary, Rest/binary>> = Buffer, + {Mp#mp{buffer= Rest}, Callback}; {exact, Skip} -> PatternLen = size(Pattern), <<DataChunk:Skip/binary, _:PatternLen/binary, Rest/binary>> = Buffer, diff --git a/src/couchdb/couch_httpd_auth.erl b/src/couchdb/couch_httpd_auth.erl index 554886ca..7023e7f3 100644 --- a/src/couchdb/couch_httpd_auth.erl +++ b/src/couchdb/couch_httpd_auth.erl @@ -16,9 +16,9 @@ -export([default_authentication_handler/1,special_test_authentication_handler/1]). -export([cookie_authentication_handler/1]). -export([null_authentication_handler/1]). +-export([proxy_authentification_handler/1]). -export([cookie_auth_header/2]). -export([handle_session_req/1]). --export([ensure_users_db_exists/1, get_user/1]). -import(couch_httpd, [header_value/2, send_json/2,send_json/4, send_method_not_allowed/2]). @@ -43,11 +43,11 @@ special_test_authentication_handler(Req) -> Req#httpd{user_ctx=#user_ctx{roles=[<<"_admin">>]}} end. -basic_username_pw(Req) -> +basic_name_pw(Req) -> AuthorizationHeader = header_value(Req, "Authorization"), case AuthorizationHeader of "Basic " ++ Base64Value -> - case string:tokens(?b2l(couch_util:decodeBase64(Base64Value)),":") of + case string:tokens(?b2l(base64:decode(Base64Value)),":") of ["_", "_"] -> % special name and pass to be logged out nil; @@ -63,20 +63,20 @@ basic_username_pw(Req) -> end. default_authentication_handler(Req) -> - case basic_username_pw(Req) of + case basic_name_pw(Req) of {User, Pass} -> - case get_user(?l2b(User)) of + case couch_auth_cache:get_user_creds(User) of nil -> throw({unauthorized, <<"Name or password is incorrect.">>}); UserProps -> - UserSalt = proplists:get_value(<<"salt">>, UserProps, <<>>), + UserSalt = couch_util:get_value(<<"salt">>, UserProps, <<>>), PasswordHash = hash_password(?l2b(Pass), UserSalt), - case proplists:get_value(<<"password_sha">>, UserProps, nil) of - ExpectedHash when ExpectedHash == PasswordHash -> + ExpectedHash = couch_util:get_value(<<"password_sha">>, UserProps, nil), + case couch_util:verify(ExpectedHash, PasswordHash) of + true -> Req#httpd{user_ctx=#user_ctx{ name=?l2b(User), - roles=proplists:get_value(<<"roles">>, UserProps, []), - user_doc={UserProps} + roles=couch_util:get_value(<<"roles">>, UserProps, []) }}; _Else -> throw({unauthorized, <<"Name or password is incorrect.">>}) @@ -99,176 +99,106 @@ default_authentication_handler(Req) -> null_authentication_handler(Req) -> Req#httpd{user_ctx=#user_ctx{roles=[<<"_admin">>]}}. -% maybe we can use hovercraft to simplify running this view query -% rename to get_user_from_users_db -get_user(UserName) -> - case couch_config:get("admins", ?b2l(UserName)) of - "-hashed-" ++ HashedPwdAndSalt -> - % the username is an admin, now check to see if there is a user doc - % which has a matching username, salt, and password_sha - [HashedPwd, Salt] = string:tokens(HashedPwdAndSalt, ","), - case get_user_props_from_db(UserName) of - nil -> - [{<<"roles">>, [<<"_admin">>]}, - {<<"salt">>, ?l2b(Salt)}, - {<<"password_sha">>, ?l2b(HashedPwd)}]; - UserProps when is_list(UserProps) -> - DocRoles = proplists:get_value(<<"roles">>, UserProps), - [{<<"roles">>, [<<"_admin">> | DocRoles]}, - {<<"salt">>, ?l2b(Salt)}, - {<<"password_sha">>, ?l2b(HashedPwd)}, - {<<"user_doc">>, {UserProps}}] - end; - Else -> - get_user_props_from_db(UserName) - end. - -get_user_props_from_db(UserName) -> - DbName = couch_config:get("couch_httpd_auth", "authentication_db"), - {ok, Db} = ensure_users_db_exists(?l2b(DbName)), - DocId = <<"org.couchdb.user:", UserName/binary>>, - try couch_httpd_db:couch_doc_open(Db, DocId, nil, []) of - #doc{}=Doc -> - {DocProps} = couch_query_servers:json_doc(Doc), - DocProps - catch - throw:Throw -> - nil - end. - -% this should handle creating the ddoc -ensure_users_db_exists(DbName) -> - case couch_db:open(DbName, [{user_ctx, #user_ctx{roles=[<<"_admin">>]}}]) of - {ok, Db} -> - ensure_auth_ddoc_exists(Db, <<"_design/_auth">>), - {ok, Db}; - _Error -> - {ok, Db} = couch_db:create(DbName, [{user_ctx, #user_ctx{roles=[<<"_admin">>]}}]), - ensure_auth_ddoc_exists(Db, <<"_design/_auth">>), - {ok, Db} +%% @doc proxy auth handler. +% +% This handler allows creation of a userCtx object from a user authenticated remotly. +% The client just pass specific headers to CouchDB and the handler create the userCtx. +% Headers name can be defined in local.ini. By thefault they are : +% +% * X-Auth-CouchDB-UserName : contain the username, (x_auth_username in +% couch_httpd_auth section) +% * X-Auth-CouchDB-Roles : contain the user roles, list of roles separated by a +% comma (x_auth_roles in couch_httpd_auth section) +% * X-Auth-CouchDB-Token : token to authenticate the authorization (x_auth_token +% in couch_httpd_auth section). This token is an hmac-sha1 created from secret key +% and username. The secret key should be the same in the client and couchdb node. s +% ecret key is the secret key in couch_httpd_auth section of ini. This token is optional +% if value of proxy_use_secret key in couch_httpd_auth section of ini isn't true. +% +proxy_authentification_handler(Req) -> + case proxy_auth_user(Req) of + nil -> Req; + Req2 -> Req2 end. -ensure_auth_ddoc_exists(Db, DDocId) -> - try couch_httpd_db:couch_doc_open(Db, DDocId, nil, []) of - _Foo -> ok - catch - _:Error -> - % create the design document - {ok, AuthDesign} = auth_design_doc(DDocId), - {ok, _Rev} = couch_db:update_doc(Db, AuthDesign, []), - ok +proxy_auth_user(Req) -> + XHeaderUserName = couch_config:get("couch_httpd_auth", "x_auth_username", + "X-Auth-CouchDB-UserName"), + XHeaderRoles = couch_config:get("couch_httpd_auth", "x_auth_roles", + "X-Auth-CouchDB-Roles"), + XHeaderToken = couch_config:get("couch_httpd_auth", "x_auth_token", + "X-Auth-CouchDB-Token"), + case header_value(Req, XHeaderUserName) of + undefined -> nil; + UserName -> + Roles = case header_value(Req, XHeaderRoles) of + undefined -> []; + Else -> + [?l2b(R) || R <- string:tokens(Else, ",")] + end, + case couch_config:get("couch_httpd_auth", "proxy_use_secret", "false") of + "true" -> + case couch_config:get("couch_httpd_auth", "secret", nil) of + nil -> + Req#httpd{user_ctx=#user_ctx{name=?l2b(UserName), roles=Roles}}; + Secret -> + ExpectedToken = couch_util:to_hex(crypto:sha_mac(Secret, UserName)), + case header_value(Req, XHeaderToken) of + Token when Token == ExpectedToken -> + Req#httpd{user_ctx=#user_ctx{name=?l2b(UserName), + roles=Roles}}; + _ -> nil + end + end; + _ -> + Req#httpd{user_ctx=#user_ctx{name=?l2b(UserName), roles=Roles}} + end end. -% add the validation function here -auth_design_doc(DocId) -> - DocProps = [ - {<<"_id">>, DocId}, - {<<"language">>,<<"javascript">>}, - {<<"views">>, - {[{<<"users">>, - {[{<<"map">>, - <<"function (doc) {\n if (doc.type == \"user\") {\n emit(doc.username, doc);\n}\n}">> - }]} - }]} - }, - { - <<"validate_doc_update">>, - <<"function(newDoc, oldDoc, userCtx) { - if (newDoc.type != 'user') { - return; - } // we only validate user docs for now - if (!newDoc.username) { - throw({forbidden : 'doc.username is required'}); - } - if (!(newDoc.roles && (typeof newDoc.roles.length != 'undefined') )) { - throw({forbidden : 'doc.roles must be an array'}); - } - if (newDoc._id != 'org.couchdb.user:'+newDoc.username) { - throw({forbidden : 'Docid must be of the form org.couchdb.user:username'}); - } - if (oldDoc) { // validate all updates - if (oldDoc.username != newDoc.username) { - throw({forbidden : 'Usernames may not be changed.'}); - } - } - if (newDoc.password_sha && !newDoc.salt) { - throw({forbidden : 'Users with password_sha must have a salt. See /_utils/script/couch.js for example code.'}); - } - if (userCtx.roles.indexOf('_admin') == -1) { // not an admin - if (oldDoc) { // validate non-admin updates - if (userCtx.name != newDoc.username) { - throw({forbidden : 'You may only update your own user document.'}); - } - // validate role updates - var oldRoles = oldDoc.roles.sort(); - var newRoles = newDoc.roles.sort(); - if (oldRoles.length != newRoles.length) { - throw({forbidden : 'Only _admin may edit roles'}); - } - for (var i=0; i < oldRoles.length; i++) { - if (oldRoles[i] != newRoles[i]) { - throw({forbidden : 'Only _admin may edit roles'}); - } - }; - } else if (newDoc.roles.length > 0) { - throw({forbidden : 'Only _admin may set roles'}); - } - } - // no system roles in users db - for (var i=0; i < newDoc.roles.length; i++) { - if (newDoc.roles[i][0] == '_') { - throw({forbidden : 'No system roles (starting with underscore) in users db.'}); - } - }; - // no system names as usernames - if (newDoc.username[0] == '_') { - throw({forbidden : 'Username may not start with underscore.'}); - } - }">> - }], - {ok, couch_doc:from_json_obj({DocProps})}. cookie_authentication_handler(#httpd{mochi_req=MochiReq}=Req) -> case MochiReq:get_cookie_value("AuthSession") of undefined -> Req; [] -> Req; - Cookie -> + Cookie -> [User, TimeStr | HashParts] = try AuthSession = couch_util:decodeBase64Url(Cookie), - [A, B | Cs] = string:tokens(?b2l(AuthSession), ":") + [_A, _B | _Cs] = string:tokens(?b2l(AuthSession), ":") catch - _:Error -> + _:_Error -> Reason = <<"Malformed AuthSession cookie. Please clear your cookies.">>, throw({bad_request, Reason}) end, % Verify expiry and hash - {NowMS, NowS, _} = erlang:now(), - CurrentTime = NowMS * 1000000 + NowS, + CurrentTime = make_cookie_time(), case couch_config:get("couch_httpd_auth", "secret", nil) of - nil -> + nil -> ?LOG_ERROR("cookie auth secret is not set",[]), Req; SecretStr -> Secret = ?l2b(SecretStr), - case get_user(?l2b(User)) of + case couch_auth_cache:get_user_creds(User) of nil -> Req; UserProps -> - UserSalt = proplists:get_value(<<"salt">>, UserProps, <<"">>), + UserSalt = couch_util:get_value(<<"salt">>, UserProps, <<"">>), FullSecret = <<Secret/binary, UserSalt/binary>>, ExpectedHash = crypto:sha_mac(FullSecret, User ++ ":" ++ TimeStr), Hash = ?l2b(string:join(HashParts, ":")), Timeout = to_int(couch_config:get("couch_httpd_auth", "timeout", 600)), ?LOG_DEBUG("timeout ~p", [Timeout]), case (catch erlang:list_to_integer(TimeStr, 16)) of - TimeStamp when CurrentTime < TimeStamp + Timeout - andalso ExpectedHash == Hash -> - TimeLeft = TimeStamp + Timeout - CurrentTime, - ?LOG_DEBUG("Successful cookie auth as: ~p", [User]), - Req#httpd{user_ctx=#user_ctx{ - name=?l2b(User), - roles=proplists:get_value(<<"roles">>, UserProps, []), - user_doc=proplists:get_value(<<"user_doc">>, UserProps, null) - }, auth={FullSecret, TimeLeft < Timeout*0.9}}; + TimeStamp when CurrentTime < TimeStamp + Timeout -> + case couch_util:verify(ExpectedHash, Hash) of + true -> + TimeLeft = TimeStamp + Timeout - CurrentTime, + ?LOG_DEBUG("Successful cookie auth as: ~p", [User]), + Req#httpd{user_ctx=#user_ctx{ + name=?l2b(User), + roles=couch_util:get_value(<<"roles">>, UserProps, []) + }, auth={FullSecret, TimeLeft < Timeout*0.9}}; + _Else -> + Req + end; _Else -> Req end @@ -285,12 +215,11 @@ cookie_auth_header(#httpd{user_ctx=#user_ctx{name=User}, auth={Secret, true}}, H % or logout handler. % The login and logout handlers need to set the AuthSession cookie % themselves. - CookieHeader = proplists:get_value("Set-Cookie", Headers, ""), + CookieHeader = couch_util:get_value("Set-Cookie", Headers, ""), Cookies = mochiweb_cookies:parse_cookie(CookieHeader), - AuthSession = proplists:get_value("AuthSession", Cookies), + AuthSession = couch_util:get_value("AuthSession", Cookies), if AuthSession == undefined -> - {NowMS, NowS, _} = erlang:now(), - TimeStamp = NowMS * 1000000 + NowS, + TimeStamp = make_cookie_time(), [cookie_auth_cookie(?b2l(User), Secret, TimeStamp)]; true -> [] @@ -322,26 +251,27 @@ ensure_cookie_auth_secret() -> handle_session_req(#httpd{method='POST', mochi_req=MochiReq}=Req) -> ReqBody = MochiReq:recv_body(), Form = case MochiReq:get_primary_header_value("content-type") of + % content type should be json "application/x-www-form-urlencoded" ++ _ -> mochiweb_util:parse_qs(ReqBody); _ -> [] end, - UserName = ?l2b(proplists:get_value("username", Form, "")), - Password = ?l2b(proplists:get_value("password", Form, "")), + UserName = ?l2b(couch_util:get_value("name", Form, "")), + Password = ?l2b(couch_util:get_value("password", Form, "")), ?LOG_DEBUG("Attempt Login: ~s",[UserName]), - User = case get_user(UserName) of + User = case couch_auth_cache:get_user_creds(UserName) of nil -> []; Result -> Result end, - UserSalt = proplists:get_value(<<"salt">>, User, <<>>), + UserSalt = couch_util:get_value(<<"salt">>, User, <<>>), PasswordHash = hash_password(Password, UserSalt), - case proplists:get_value(<<"password_sha">>, User, nil) of - ExpectedHash when ExpectedHash == PasswordHash -> + ExpectedHash = couch_util:get_value(<<"password_sha">>, User, nil), + case couch_util:verify(ExpectedHash, PasswordHash) of + true -> % setup the session cookie Secret = ?l2b(ensure_cookie_auth_secret()), - {NowMS, NowS, _} = erlang:now(), - CurrentTime = NowMS * 1000000 + NowS, + CurrentTime = make_cookie_time(), Cookie = cookie_auth_cookie(?b2l(UserName), <<Secret/binary, UserSalt/binary>>, CurrentTime), % TODO document the "next" feature in Futon {Code, Headers} = case couch_httpd:qs_value(Req, "next", nil) of @@ -353,9 +283,8 @@ handle_session_req(#httpd{method='POST', mochi_req=MochiReq}=Req) -> send_json(Req#httpd{req_body=ReqBody}, Code, Headers, {[ {ok, true}, - {name, proplists:get_value(<<"username">>, User, null)}, - {roles, proplists:get_value(<<"roles">>, User, [])}, - {user_doc, proplists:get_value(<<"user_doc">>, User, null)} + {name, couch_util:get_value(<<"name">>, User, null)}, + {roles, couch_util:get_value(<<"roles">>, User, [])} ]}); _Else -> % clear the session @@ -363,6 +292,7 @@ handle_session_req(#httpd{method='POST', mochi_req=MochiReq}=Req) -> send_json(Req, 401, [Cookie], {[{error, <<"unauthorized">>},{reason, <<"Name or password is incorrect.">>}]}) end; % get user info +% GET /_session handle_session_req(#httpd{method='GET', user_ctx=UserCtx}=Req) -> Name = UserCtx#user_ctx.name, ForceLogin = couch_httpd:qs_value(Req, "basic", "false"), @@ -371,15 +301,20 @@ handle_session_req(#httpd{method='GET', user_ctx=UserCtx}=Req) -> throw({unauthorized, <<"Please login.">>}); {Name, _} -> send_json(Req, {[ + % remove this ok {ok, true}, - {name, Name}, - {roles, UserCtx#user_ctx.roles}, + {<<"userCtx">>, {[ + {name, Name}, + {roles, UserCtx#user_ctx.roles} + ]}}, {info, {[ - {user_db, ?l2b(couch_config:get("couch_httpd_auth", "authentication_db"))}, - {handlers, [?l2b(H) || H <- couch_httpd:make_fun_spec_strs( + {authentication_db, ?l2b(couch_config:get("couch_httpd_auth", "authentication_db"))}, + {authentication_handlers, [auth_name(H) || H <- couch_httpd:make_fun_spec_strs( couch_config:get("httpd", "authentication_handlers"))]} - ] ++ maybe_value(authenticated, UserCtx#user_ctx.handler)}} - ] ++ maybe_value(user_doc, UserCtx#user_ctx.user_doc)}) + ] ++ maybe_value(authenticated, UserCtx#user_ctx.handler, fun(Handler) -> + auth_name(?b2l(Handler)) + end)}} + ]}) end; % logout by deleting the session handle_session_req(#httpd{method='DELETE'}=Req) -> @@ -394,12 +329,21 @@ handle_session_req(#httpd{method='DELETE'}=Req) -> handle_session_req(Req) -> send_method_not_allowed(Req, "GET,HEAD,POST,DELETE"). -maybe_value(Key, undefined) -> []; -maybe_value(Key, Else) -> [{Key, Else}]. +maybe_value(_Key, undefined, _Fun) -> []; +maybe_value(Key, Else, Fun) -> + [{Key, Fun(Else)}]. + +auth_name(String) when is_list(String) -> + [_,_,_,_,_,Name|_] = re:split(String, "[\\W_]", [{return, list}]), + ?l2b(Name). to_int(Value) when is_binary(Value) -> - to_int(?b2l(Value)); + to_int(?b2l(Value)); to_int(Value) when is_list(Value) -> list_to_integer(Value); to_int(Value) when is_integer(Value) -> Value. + +make_cookie_time() -> + {NowMS, NowS, _} = erlang:now(), + NowMS * 1000000 + NowS. diff --git a/src/couchdb/couch_httpd_db.erl b/src/couchdb/couch_httpd_db.erl index 9233e953..7b09bf57 100644 --- a/src/couchdb/couch_httpd_db.erl +++ b/src/couchdb/couch_httpd_db.erl @@ -21,7 +21,7 @@ -import(couch_httpd, [send_json/2,send_json/3,send_json/4,send_method_not_allowed/2, start_json_response/2,start_json_response/3, - send_chunk/2,end_json_response/1, + send_chunk/2,last_chunk/1,end_json_response/1, start_chunked_response/3, absolute_uri/2, send/2, start_response_length/4]). @@ -55,205 +55,74 @@ handle_request(#httpd{path_parts=[DbName|RestParts],method=Method, do_db_req(Req, Handler) end. -get_changes_timeout(Req, Resp) -> - DefaultTimeout = list_to_integer( - couch_config:get("httpd", "changes_timeout", "60000")), - case couch_httpd:qs_value(Req, "heartbeat") of - undefined -> - case couch_httpd:qs_value(Req, "timeout") of - undefined -> - {DefaultTimeout, fun() -> stop end}; - TimeoutList -> - {lists:min([DefaultTimeout, list_to_integer(TimeoutList)]), - fun() -> stop end} - end; - "true" -> - {DefaultTimeout, fun() -> send_chunk(Resp, "\n"), ok end}; - TimeoutList -> - {lists:min([DefaultTimeout, list_to_integer(TimeoutList)]), - fun() -> send_chunk(Resp, "\n"), ok end} - end. - - -start_sending_changes(_Resp, "continuous") -> - ok; -start_sending_changes(Resp, _Else) -> - send_chunk(Resp, "{\"results\":[\n"). - -handle_changes_req(#httpd{method='GET',path_parts=[DbName|_]}=Req, Db) -> - FilterFun = make_filter_fun(Req, Db), - {ok, Info} = couch_db:get_db_info(Db), - Seq = proplists:get_value(update_seq, Info), - {Dir, StartSeq} = case couch_httpd:qs_value(Req, "descending", "false") of - "false" -> - {fwd, list_to_integer(couch_httpd:qs_value(Req, "since", "0"))}; - "true" -> - {rev, Seq}; - _Bad -> throw({bad_request, "descending must be true or false"}) +handle_changes_req(#httpd{method='GET'}=Req, Db) -> + MakeCallback = fun(Resp) -> + fun({change, Change, _}, "continuous") -> + send_chunk(Resp, [?JSON_ENCODE(Change) | "\n"]); + ({change, Change, Prepend}, _) -> + send_chunk(Resp, [Prepend, ?JSON_ENCODE(Change)]); + (start, "continuous") -> + ok; + (start, _) -> + send_chunk(Resp, "{\"results\":[\n"); + ({stop, EndSeq}, "continuous") -> + send_chunk( + Resp, + [?JSON_ENCODE({[{<<"last_seq">>, EndSeq}]}) | "\n"] + ), + end_json_response(Resp); + ({stop, EndSeq}, _) -> + send_chunk( + Resp, + io_lib:format("\n],\n\"last_seq\":~w}\n", [EndSeq]) + ), + end_json_response(Resp); + (timeout, _) -> + send_chunk(Resp, "\n") + end end, - Limit = list_to_integer(couch_httpd:qs_value(Req, "limit", "1000000000000000")), - ResponseType = couch_httpd:qs_value(Req, "feed", "normal"), - if ResponseType == "continuous" orelse ResponseType == "longpoll" -> - {ok, Resp} = start_json_response(Req, 200), - start_sending_changes(Resp, ResponseType), - - Self = self(), - {ok, Notify} = couch_db_update_notifier:start_link( - fun({_, DbName0}) when DbName0 == DbName -> - Self ! db_updated; - (_) -> - ok - end), - {Timeout, TimeoutFun} = get_changes_timeout(Req, Resp), - couch_stats_collector:track_process_count(Self, - {httpd, clients_requesting_changes}), - try - keep_sending_changes(Req, Resp, Db, StartSeq, <<"">>, Timeout, - TimeoutFun, ResponseType, Limit, FilterFun) - after - couch_db_update_notifier:stop(Notify), - get_rest_db_updated() % clean out any remaining update messages - end; - true -> + ChangesArgs = parse_changes_query(Req), + ChangesFun = couch_changes:handle_changes(ChangesArgs, Req, Db), + WrapperFun = case ChangesArgs#changes_args.feed of + "normal" -> + {ok, Info} = couch_db:get_db_info(Db), CurrentEtag = couch_httpd:make_etag(Info), - couch_httpd:etag_respond(Req, CurrentEtag, fun() -> - % send the etag - {ok, Resp} = start_json_response(Req, 200, [{"Etag", CurrentEtag}]), - start_sending_changes(Resp, ResponseType), - {ok, {_, LastSeq, _Prepend, _, _, _, _, _}} = - send_changes(Req, Resp, Db, Dir, StartSeq, <<"">>, "normal", - Limit, FilterFun), - end_sending_changes(Resp, LastSeq, ResponseType) - end) - end; + fun(FeedChangesFun) -> + couch_httpd:etag_respond( + Req, + CurrentEtag, + fun() -> + {ok, Resp} = couch_httpd:start_json_response( + Req, 200, [{"Etag", CurrentEtag}] + ), + FeedChangesFun(MakeCallback(Resp)) + end + ) + end; + _ -> + % "longpoll" or "continuous" + {ok, Resp} = couch_httpd:start_json_response(Req, 200), + fun(FeedChangesFun) -> + FeedChangesFun(MakeCallback(Resp)) + end + end, + couch_stats_collector:track_process_count( + {httpd, clients_requesting_changes} + ), + WrapperFun(ChangesFun); handle_changes_req(#httpd{path_parts=[_,<<"_changes">>]}=Req, _Db) -> send_method_not_allowed(Req, "GET,HEAD"). -% waits for a db_updated msg, if there are multiple msgs, collects them. -wait_db_updated(Timeout, TimeoutFun) -> - receive db_updated -> get_rest_db_updated() - after Timeout -> - case TimeoutFun() of - ok -> wait_db_updated(Timeout, TimeoutFun); - stop -> stop - end - end. - -get_rest_db_updated() -> - receive db_updated -> get_rest_db_updated() - after 0 -> updated - end. - -end_sending_changes(Resp, EndSeq, "continuous") -> - send_chunk(Resp, [?JSON_ENCODE({[{<<"last_seq">>, EndSeq}]}) | "\n"]), - end_json_response(Resp); -end_sending_changes(Resp, EndSeq, _Else) -> - send_chunk(Resp, io_lib:format("\n],\n\"last_seq\":~w}\n", [EndSeq])), - end_json_response(Resp). - -keep_sending_changes(#httpd{user_ctx=UserCtx,path_parts=[DbName|_]}=Req, Resp, - Db, StartSeq, Prepend, Timeout, TimeoutFun, ResponseType, Limit, Filter) -> - {ok, {_, EndSeq, Prepend2, _, _, _, NewLimit, _}} = send_changes(Req, Resp, Db, fwd, StartSeq, - Prepend, ResponseType, Limit, Filter), - couch_db:close(Db), - if - Limit > NewLimit, ResponseType == "longpoll" -> - end_sending_changes(Resp, EndSeq, ResponseType); - true -> - case wait_db_updated(Timeout, TimeoutFun) of - updated -> - case couch_db:open(DbName, [{user_ctx, UserCtx}]) of - {ok, Db2} -> - keep_sending_changes(Req, Resp, Db2, EndSeq, Prepend2, Timeout, - TimeoutFun, ResponseType, NewLimit, Filter); - _Else -> - end_sending_changes(Resp, EndSeq, ResponseType) - end; - stop -> - end_sending_changes(Resp, EndSeq, ResponseType) - end - end. - -changes_enumerator(DocInfos, {Db, _, _, FilterFun, Resp, "continuous", Limit, IncludeDocs}) -> - [#doc_info{id=Id, high_seq=Seq, revs=[#rev_info{deleted=Del,rev=Rev}|_]}|_] = DocInfos, - Results0 = FilterFun(DocInfos), - Results = [Result || Result <- Results0, Result /= null], - Go = if Limit =< 1 -> stop; true -> ok end, - case Results of - [] -> - {Go, {Db, Seq, nil, FilterFun, Resp, "continuous", Limit, IncludeDocs}}; - _ -> - send_chunk(Resp, [?JSON_ENCODE(changes_row(Db, Seq, Id, Del, Results, Rev, IncludeDocs)) - |"\n"]), - {Go, {Db, Seq, nil, FilterFun, Resp, "continuous", Limit-1, IncludeDocs}} - end; -changes_enumerator(DocInfos, {Db, _, Prepend, FilterFun, Resp, _, Limit, IncludeDocs}) -> - [#doc_info{id=Id, high_seq=Seq, revs=[#rev_info{deleted=Del,rev=Rev}|_]}|_] = DocInfos, - Results0 = FilterFun(DocInfos), - Results = [Result || Result <- Results0, Result /= null], - Go = if Limit =< 1 -> stop; true -> ok end, - case Results of - [] -> - {Go, {Db, Seq, Prepend, FilterFun, Resp, nil, Limit, IncludeDocs}}; - _ -> - send_chunk(Resp, [Prepend, ?JSON_ENCODE( - changes_row(Db, Seq, Id, Del, Results, Rev, IncludeDocs))]), - {Go, {Db, Seq, <<",\n">>, FilterFun, Resp, nil, Limit-1, IncludeDocs}} - end. - -changes_row(Db, Seq, Id, Del, Results, Rev, true) -> - {[{seq,Seq},{id,Id},{changes,Results}] ++ deleted_item(Del) ++ - couch_httpd_view:doc_member(Db, {Id, Rev})}; -changes_row(_, Seq, Id, Del, Results, _, false) -> - {[{seq,Seq},{id,Id},{changes,Results}] ++ deleted_item(Del)}. - -deleted_item(true) -> [{deleted,true}]; -deleted_item(_) -> []. - -send_changes(Req, Resp, Db, Dir, StartSeq, Prepend, ResponseType, Limit, FilterFun) -> - Style = list_to_existing_atom( - couch_httpd:qs_value(Req, "style", "main_only")), - IncludeDocs = list_to_existing_atom( - couch_httpd:qs_value(Req, "include_docs", "false")), - couch_db:changes_since(Db, Style, StartSeq, fun changes_enumerator/2, - [{dir, Dir}], {Db, StartSeq, Prepend, FilterFun, Resp, ResponseType, Limit, IncludeDocs}). - -make_filter_fun(Req, Db) -> - Filter = couch_httpd:qs_value(Req, "filter", ""), - case [list_to_binary(couch_httpd:unquote(Part)) - || Part <- string:tokens(Filter, "/")] of - [] -> - fun(DocInfos) -> - % doing this as a batch is more efficient for external filters - [{[{rev, couch_doc:rev_to_str(Rev)}]} || - #doc_info{revs=[#rev_info{rev=Rev}|_]} <- DocInfos] - end; - [DName, FName] -> - DesignId = <<"_design/", DName/binary>>, - DDoc = couch_httpd_db:couch_doc_open(Db, DesignId, nil, []), - % validate that the ddoc has the filter fun - #doc{body={Props}} = DDoc, - couch_util:get_nested_json_value({Props}, [<<"filters">>, FName]), - fun(DocInfos) -> - Docs = [Doc || {ok, Doc} <- [ - {ok, Doc} = couch_db:open_doc(Db, DInfo, [deleted]) - || DInfo <- DocInfos]], - {ok, Passes} = couch_query_servers:filter_docs(Req, Db, DDoc, FName, Docs), - [{[{rev, couch_doc:rev_to_str(Rev)}]} - || #doc_info{revs=[#rev_info{rev=Rev}|_]} <- DocInfos, - Pass <- Passes, Pass == true] - end; - _Else -> - throw({bad_request, - "filter parameter must be of the form `designname/filtername`"}) - end. - -handle_compact_req(#httpd{method='POST',path_parts=[DbName,_,Id|_]}=Req, _Db) -> +handle_compact_req(#httpd{method='POST',path_parts=[DbName,_,Id|_]}=Req, Db) -> + ok = couch_db:check_is_admin(Db), + couch_httpd:validate_ctype(Req, "application/json"), ok = couch_view_compactor:start_compact(DbName, Id), send_json(Req, 202, {[{ok, true}]}); handle_compact_req(#httpd{method='POST'}=Req, Db) -> + ok = couch_db:check_is_admin(Db), + couch_httpd:validate_ctype(Req, "application/json"), ok = couch_db:start_compact(Db), send_json(Req, 202, {[{ok, true}]}); @@ -262,6 +131,8 @@ handle_compact_req(Req, _Db) -> handle_view_cleanup_req(#httpd{method='POST'}=Req, Db) -> % delete unreferenced index files + ok = couch_db:check_is_admin(Db), + couch_httpd:validate_ctype(Req, "application/json"), ok = couch_view:cleanup_index_files(Db), send_json(Req, 202, {[{ok, true}]}); @@ -276,7 +147,9 @@ handle_design_req(#httpd{ % load ddoc DesignId = <<"_design/", DesignName/binary>>, DDoc = couch_httpd_db:couch_doc_open(Db, DesignId, nil, []), - Handler = couch_util:dict_find(Action, DesignUrlHandlers, fun db_req/2), + Handler = couch_util:dict_find(Action, DesignUrlHandlers, fun(_, _, _) -> + throw({not_found, <<"missing handler: ", Action/binary>>}) + end), Handler(Req, Db, DDoc); handle_design_req(Req, Db) -> @@ -298,23 +171,13 @@ handle_design_info_req(Req, _Db, _DDoc) -> create_db_req(#httpd{user_ctx=UserCtx}=Req, DbName) -> ok = couch_httpd:verify_is_server_admin(Req), - LDbName = ?b2l(DbName), - case couch_config:get("couch_httpd_auth", "authentication_db") of - LDbName -> - % make sure user's db always has the auth ddoc - {ok, Db} = couch_httpd_auth:ensure_users_db_exists(DbName), - couch_db:close(Db), - DbUrl = absolute_uri(Req, "/" ++ couch_util:url_encode(DbName)), - send_json(Req, 201, [{"Location", DbUrl}], {[{ok, true}]}); - _Else -> - case couch_server:create(DbName, [{user_ctx, UserCtx}]) of - {ok, Db} -> - couch_db:close(Db), - DbUrl = absolute_uri(Req, "/" ++ couch_util:url_encode(DbName)), - send_json(Req, 201, [{"Location", DbUrl}], {[{ok, true}]}); - Error -> - throw(Error) - end + case couch_server:create(DbName, [{user_ctx, UserCtx}]) of + {ok, Db} -> + couch_db:close(Db), + DbUrl = absolute_uri(Req, "/" ++ couch_util:url_encode(DbName)), + send_json(Req, 201, [{"Location", DbUrl}], {[{ok, true}]}); + Error -> + throw(Error) end. delete_db_req(#httpd{user_ctx=UserCtx}=Req, DbName) -> @@ -327,15 +190,6 @@ delete_db_req(#httpd{user_ctx=UserCtx}=Req, DbName) -> end. do_db_req(#httpd{user_ctx=UserCtx,path_parts=[DbName|_]}=Req, Fun) -> - LDbName = ?b2l(DbName), - % I hope this lookup is cheap. - case couch_config:get("couch_httpd_auth", "authentication_db") of - LDbName -> - % make sure user's db always has the auth ddoc - {ok, ADb} = couch_httpd_auth:ensure_users_db_exists(DbName), - couch_db:close(ADb); - _Else -> ok - end, case couch_db:open(DbName, [{user_ctx, UserCtx}]) of {ok, Db} -> try @@ -352,6 +206,7 @@ db_req(#httpd{method='GET',path_parts=[_DbName]}=Req, Db) -> send_json(Req, {DbInfo}); db_req(#httpd{method='POST',path_parts=[DbName]}=Req, Db) -> + couch_httpd:validate_ctype(Req, "application/json"), Doc = couch_doc:from_json_obj(couch_httpd:json_body(Req)), Doc2 = case Doc#doc.id of <<"">> -> @@ -392,6 +247,7 @@ db_req(#httpd{path_parts=[_DbName]}=Req, _Db) -> send_method_not_allowed(Req, "DELETE,GET,HEAD,POST"); db_req(#httpd{method='POST',path_parts=[_,<<"_ensure_full_commit">>]}=Req, Db) -> + couch_httpd:validate_ctype(Req, "application/json"), UpdateSeq = couch_db:get_update_seq(Db), CommittedSeq = couch_db:get_committed_update_seq(Db), {ok, StartTime} = @@ -404,7 +260,6 @@ db_req(#httpd{method='POST',path_parts=[_,<<"_ensure_full_commit">>]}=Req, Db) - throw({bad_request, "can't do a full commit ahead of current update_seq"}); RequiredSeq > CommittedSeq -> - % user asked for an explicit sequence, don't commit any batches couch_db:ensure_full_commit(Db); true -> {ok, Db#db.instance_start_time} @@ -420,8 +275,9 @@ db_req(#httpd{path_parts=[_,<<"_ensure_full_commit">>]}=Req, _Db) -> db_req(#httpd{method='POST',path_parts=[_,<<"_bulk_docs">>]}=Req, Db) -> couch_stats_collector:increment({httpd, bulk_requests}), + couch_httpd:validate_ctype(Req, "application/json"), {JsonProps} = couch_httpd:json_body_obj(Req), - DocsArray = proplists:get_value(<<"docs">>, JsonProps), + DocsArray = couch_util:get_value(<<"docs">>, JsonProps), case couch_httpd:header_value(Req, "X-Couch-Full-Commit") of "true" -> Options = [full_commit]; @@ -430,7 +286,7 @@ db_req(#httpd{method='POST',path_parts=[_,<<"_bulk_docs">>]}=Req, Db) -> _ -> Options = [] end, - case proplists:get_value(<<"new_edits">>, JsonProps, true) of + case couch_util:get_value(<<"new_edits">>, JsonProps, true) of true -> Docs = lists:map( fun({ObjProps} = JsonObj) -> @@ -440,7 +296,7 @@ db_req(#httpd{method='POST',path_parts=[_,<<"_bulk_docs">>]}=Req, Db) -> <<>> -> couch_uuids:new(); Id0 -> Id0 end, - case proplists:get_value(<<"_rev">>, ObjProps) of + case couch_util:get_value(<<"_rev">>, ObjProps) of undefined -> Revs = {0, []}; Rev -> @@ -451,7 +307,7 @@ db_req(#httpd{method='POST',path_parts=[_,<<"_bulk_docs">>]}=Req, Db) -> end, DocsArray), Options2 = - case proplists:get_value(<<"all_or_nothing">>, JsonProps) of + case couch_util:get_value(<<"all_or_nothing">>, JsonProps) of true -> [all_or_nothing|Options]; _ -> Options end, @@ -467,7 +323,11 @@ db_req(#httpd{method='POST',path_parts=[_,<<"_bulk_docs">>]}=Req, Db) -> send_json(Req, 417, ErrorsJson) end; false -> - Docs = [couch_doc:from_json_obj(JsonObj) || JsonObj <- DocsArray], + Docs = lists:map(fun(JsonObj) -> + Doc = couch_doc:from_json_obj(JsonObj), + validate_attachment_names(Doc), + Doc + end, DocsArray), {ok, Errors} = couch_db:update_docs(Db, Docs, Options, replicated_changes), ErrorsJson = lists:map(fun update_doc_result_to_json/1, Errors), @@ -477,6 +337,7 @@ db_req(#httpd{path_parts=[_,<<"_bulk_docs">>]}=Req, _Db) -> send_method_not_allowed(Req, "POST"); db_req(#httpd{method='POST',path_parts=[_,<<"_purge">>]}=Req, Db) -> + couch_httpd:validate_ctype(Req, "application/json"), {IdsRevs} = couch_httpd:json_body_obj(Req), IdsRevs2 = [{Id, couch_doc:parse_revs(Revs)} || {Id, Revs} <- IdsRevs], @@ -496,7 +357,7 @@ db_req(#httpd{method='GET',path_parts=[_,<<"_all_docs">>]}=Req, Db) -> db_req(#httpd{method='POST',path_parts=[_,<<"_all_docs">>]}=Req, Db) -> {Fields} = couch_httpd:json_body_obj(Req), - case proplists:get_value(<<"keys">>, Fields, nil) of + case couch_util:get_value(<<"keys">>, Fields, nil) of nil -> ?LOG_DEBUG("POST to _all_docs with no keys member.", []), all_docs_view(Req, Db, nil); @@ -521,39 +382,36 @@ 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='POST',path_parts=[_,<<"_revs_diff">>]}=Req, Db) -> {JsonDocIdRevs} = couch_httpd:json_body_obj(Req), - JsonDocIdRevs2 = + JsonDocIdRevs2 = [{Id, couch_doc:parse_revs(RevStrs)} || {Id, RevStrs} <- JsonDocIdRevs], {ok, Results} = couch_db:get_missing_revs(Db, JsonDocIdRevs2), - Results2 = + Results2 = lists:map(fun({Id, MissingRevs, PossibleAncestors}) -> {Id, - {[{missing, couch_doc:revs_to_strs(MissingRevs)}] ++ + {[{missing, couch_doc:revs_to_strs(MissingRevs)}] ++ if PossibleAncestors == [] -> []; - true -> - [{possible_ancestors, + true -> + [{possible_ancestors, couch_doc:revs_to_strs(PossibleAncestors)}] end}} end, Results), - send_json(Req, {[{missing, {Results2}}]}); + send_json(Req, {Results2}); db_req(#httpd{path_parts=[_,<<"_revs_diff">>]}=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), +db_req(#httpd{method='PUT',path_parts=[_,<<"_security">>]}=Req, Db) -> + SecObj = couch_httpd:json_body(Req), + ok = couch_db:set_security(Db, SecObj), 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{method='GET',path_parts=[_,<<"_security">>]}=Req, Db) -> + send_json(Req, couch_db:get_security(Db)); -db_req(#httpd{path_parts=[_,<<"_admins">>]}=Req, _Db) -> +db_req(#httpd{path_parts=[_,<<"_security">>]}=Req, _Db) -> send_method_not_allowed(Req, "PUT,GET"); db_req(#httpd{method='PUT',path_parts=[_,<<"_revs_limit">>]}=Req, @@ -620,7 +478,7 @@ all_docs_view(Req, Db, Keys) -> CurrentEtag = couch_httpd:make_etag(Info), couch_httpd:etag_respond(Req, CurrentEtag, fun() -> - TotalRowCount = proplists:get_value(doc_count, Info), + TotalRowCount = couch_util:get_value(doc_count, Info), StartId = if is_binary(StartKey) -> StartKey; true -> StartDocId end, @@ -628,10 +486,16 @@ all_docs_view(Req, Db, Keys) -> true -> EndDocId end, FoldAccInit = {Limit, SkipCount, undefined, []}, - + UpdateSeq = couch_db:get_update_seq(Db), + JsonParams = case couch_httpd:qs_value(Req, "update_seq") of + "true" -> + [{update_seq, UpdateSeq}]; + _Else -> + [] + end, case Keys of nil -> - FoldlFun = couch_httpd_view:make_view_fold_fun(Req, QueryArgs, CurrentEtag, Db, + FoldlFun = couch_httpd_view:make_view_fold_fun(Req, QueryArgs, CurrentEtag, Db, UpdateSeq, TotalRowCount, #view_fold_helper_funs{ reduce_count = fun couch_db:enum_docs_reduce_to_count/1 }), @@ -646,9 +510,9 @@ all_docs_view(Req, Db, Keys) -> {ok, LastOffset, FoldResult} = couch_db:enum_docs(Db, AdapterFun, FoldAccInit, [{start_key, StartId}, {dir, Dir}, {if Inclusive -> end_key; true -> end_key_gt end, EndId}]), - couch_httpd_view:finish_view_fold(Req, TotalRowCount, LastOffset, FoldResult); + couch_httpd_view:finish_view_fold(Req, TotalRowCount, LastOffset, FoldResult, JsonParams); _ -> - FoldlFun = couch_httpd_view:make_view_fold_fun(Req, QueryArgs, CurrentEtag, Db, + FoldlFun = couch_httpd_view:make_view_fold_fun(Req, QueryArgs, CurrentEtag, Db, UpdateSeq, TotalRowCount, #view_fold_helper_funs{ reduce_count = fun(Offset) -> Offset end }), @@ -675,7 +539,7 @@ all_docs_view(Req, Db, Keys) -> {_, FoldAcc2} = FoldlFun(Doc, 0, FoldAcc), FoldAcc2 end, FoldAccInit, Keys), - couch_httpd_view:finish_view_fold(Req, TotalRowCount, 0, FoldResult) + couch_httpd_view:finish_view_fold(Req, TotalRowCount, 0, FoldResult, JsonParams) end end). @@ -684,11 +548,11 @@ db_doc_req(#httpd{method='DELETE'}=Req, Db, DocId) -> couch_doc_open(Db, DocId, nil, []), case couch_httpd:qs_value(Req, "rev") of undefined -> - update_doc(Req, Db, DocId, + update_doc(Req, Db, DocId, couch_doc_from_req(Req, DocId, {[{<<"_deleted">>,true}]})); Rev -> - update_doc(Req, Db, DocId, - couch_doc_from_req(Req, DocId, + update_doc(Req, Db, DocId, + couch_doc_from_req(Req, DocId, {[{<<"_rev">>, ?l2b(Rev)},{<<"_deleted">>,true}]})) end; @@ -701,54 +565,59 @@ db_doc_req(#httpd{method='GET'}=Req, Db, DocId) -> } = parse_doc_query(Req), case Revs of [] -> - Doc = couch_doc_open(Db, DocId, Rev, Options), Options2 = if AttsSince /= nil -> - RevPos = find_ancestor_rev_pos(Doc#doc.revs, AttsSince), - [{atts_after_revpos, RevPos} | Options]; + [{atts_since, AttsSince}, attachments | Options]; true -> Options end, + Doc = couch_doc_open(Db, DocId, Rev, Options2), send_doc(Req, Doc, Options2); _ -> {ok, Results} = couch_db:open_doc_revs(Db, DocId, Revs, Options), - {ok, Resp} = start_json_response(Req, 200), - send_chunk(Resp, "["), - % We loop through the docs. The first time through the separator - % is whitespace, then a comma on subsequent iterations. - lists:foldl( - fun(Result, AccSeparator) -> - case Result of - {ok, Doc} -> - JsonDoc = couch_doc:to_json_obj(Doc, Options), - Json = ?JSON_ENCODE({[{ok, JsonDoc}]}), - send_chunk(Resp, AccSeparator ++ Json); - {{not_found, missing}, RevId} -> - RevStr = couch_doc:rev_to_str(RevId), - Json = ?JSON_ENCODE({[{"missing", RevStr}]}), - send_chunk(Resp, AccSeparator ++ Json) + AcceptedTypes = case couch_httpd:header_value(Req, "Accept") of + undefined -> []; + AcceptHeader -> string:tokens(AcceptHeader, ", ") + end, + case lists:member("multipart/mixed", AcceptedTypes) of + false -> + {ok, Resp} = start_json_response(Req, 200), + send_chunk(Resp, "["), + % We loop through the docs. The first time through the separator + % is whitespace, then a comma on subsequent iterations. + lists:foldl( + fun(Result, AccSeparator) -> + case Result of + {ok, Doc} -> + JsonDoc = couch_doc:to_json_obj(Doc, Options), + Json = ?JSON_ENCODE({[{ok, JsonDoc}]}), + send_chunk(Resp, AccSeparator ++ Json); + {{not_found, missing}, RevId} -> + RevStr = couch_doc:rev_to_str(RevId), + Json = ?JSON_ENCODE({[{"missing", RevStr}]}), + send_chunk(Resp, AccSeparator ++ Json) + end, + "," % AccSeparator now has a comma end, - "," % AccSeparator now has a comma - end, - "", Results), - send_chunk(Resp, "]"), - end_json_response(Resp) + "", Results), + send_chunk(Resp, "]"), + end_json_response(Resp); + true -> + send_docs_multipart(Req, Results, Options) + end end; + db_doc_req(#httpd{method='POST'}=Req, Db, DocId) -> + couch_httpd:validate_referer(Req), couch_doc:validate_docid(DocId), - case couch_httpd:header_value(Req, "Content-Type") of - "multipart/form-data" ++ _Rest -> - ok; - _Else -> - throw({bad_ctype, <<"Invalid Content-Type header for form upload">>}) - end, + couch_httpd:validate_ctype(Req, "multipart/form-data"), Form = couch_httpd:parse_form(Req), case proplists:is_defined("_doc", Form) of true -> - Json = ?JSON_DECODE(proplists:get_value("_doc", Form)), + Json = ?JSON_DECODE(couch_util:get_value("_doc", Form)), Doc = couch_doc_from_req(Req, DocId, Json); false -> - Rev = couch_doc:parse_rev(list_to_binary(proplists:get_value("_rev", Form))), + Rev = couch_doc:parse_rev(list_to_binary(couch_util:get_value("_rev", Form))), {ok, [{ok, Doc}]} = couch_db:open_doc_revs(Db, DocId, [Rev], []) end, UpdatedAtts = [ @@ -785,13 +654,13 @@ db_doc_req(#httpd{method='PUT'}=Req, Db, DocId) -> Loc = absolute_uri(Req, "/" ++ ?b2l(Db#db.name) ++ "/" ++ ?b2l(DocId)), RespHeaders = [{"Location", Loc}], - case couch_httpd:header_value(Req, "Content-Type") of - ("multipart/related" ++ _Rest) = ContentType-> - Doc0 = couch_doc:doc_from_multi_part_stream(ContentType, + case couch_util:to_list(couch_httpd:header_value(Req, "Content-Type")) of + ("multipart/related;" ++ _) = ContentType -> + {ok, Doc0} = couch_doc:doc_from_multi_part_stream(ContentType, fun() -> receive_request_data(Req) end), Doc = couch_doc_from_req(Req, DocId, Doc0), update_doc(Req, Db, DocId, Doc, RespHeaders, UpdateType); - _ -> + _Else -> case couch_httpd:qs_value(Req, "batch") of "ok" -> % batch @@ -810,7 +679,8 @@ db_doc_req(#httpd{method='PUT'}=Req, Db, DocId) -> ]}); _Normal -> % normal - Doc = couch_doc_from_req(Req, DocId, couch_httpd:json_body(Req)), + Body = couch_httpd:json_body(Req), + Doc = couch_doc_from_req(Req, DocId, Body), update_doc(Req, Db, DocId, Doc, RespHeaders, UpdateType) end end; @@ -835,24 +705,13 @@ db_doc_req(#httpd{method='COPY'}=Req, Db, SourceDocId) -> db_doc_req(Req, _Db, _DocId) -> send_method_not_allowed(Req, "DELETE,GET,HEAD,POST,PUT,COPY"). -find_ancestor_rev_pos({_, []}, _AttsSinceRevs) -> - 0; -find_ancestor_rev_pos(_DocRevs, []) -> - 0; -find_ancestor_rev_pos({RevPos, [RevId|Rest]}, AttsSinceRevs) -> - case lists:member({RevPos, RevId}, AttsSinceRevs) of - true -> - RevPos; - false -> - find_ancestor_rev_pos({RevPos - 1, Rest}, AttsSinceRevs) - end. send_doc(Req, Doc, Options) -> case Doc#doc.meta of [] -> DiskEtag = couch_httpd:doc_etag(Doc), % output etag only when we have no meta - couch_httpd:etag_respond(Req, DiskEtag, fun() -> + couch_httpd:etag_respond(Req, DiskEtag, fun() -> send_doc_efficiently(Req, Doc, [{"Etag", DiskEtag}], Options) end); _ -> @@ -863,8 +722,7 @@ send_doc(Req, Doc, Options) -> send_doc_efficiently(Req, #doc{atts=[]}=Doc, Headers, Options) -> send_json(Req, 200, Headers, couch_doc:to_json_obj(Doc, Options)); send_doc_efficiently(Req, #doc{atts=Atts}=Doc, Headers, Options) -> - case lists:member(attachments, Options) orelse - proplists:is_defined(atts_after_revpos, Options) of + case lists:member(attachments, Options) of true -> AcceptedTypes = case couch_httpd:header_value(Req, "Accept") of undefined -> []; @@ -872,25 +730,51 @@ send_doc_efficiently(Req, #doc{atts=Atts}=Doc, Headers, Options) -> end, case lists:member("multipart/related", AcceptedTypes) of false -> - send_json(Req, 200, [], couch_doc:to_json_obj(Doc, Options)); + send_json(Req, 200, Headers, couch_doc:to_json_obj(Doc, Options)); true -> Boundary = couch_uuids:random(), - JsonBytes = ?JSON_ENCODE(couch_doc:to_json_obj(Doc, [follows|Options])), - AttsSinceRevPos = proplists:get_value(atts_after_revpos, Options, 0), - Len = couch_doc:len_doc_to_multi_part_stream(Boundary,JsonBytes,Atts, - AttsSinceRevPos), - CType = {<<"Content-Type">>, - <<"multipart/related; boundary=\"", Boundary/binary, "\"">>}, + JsonBytes = ?JSON_ENCODE(couch_doc:to_json_obj(Doc, + [attachments, follows|Options])), + {ContentType, Len} = couch_doc:len_doc_to_multi_part_stream( + Boundary,JsonBytes, Atts,false), + CType = {<<"Content-Type">>, ContentType}, {ok, Resp} = start_response_length(Req, 200, [CType|Headers], Len), couch_doc:doc_to_multi_part_stream(Boundary,JsonBytes,Atts, - AttsSinceRevPos, - fun(Data) -> couch_httpd:send(Resp, Data) end) + fun(Data) -> couch_httpd:send(Resp, Data) end, false) end; false -> - send_json(Req, 200, [], couch_doc:to_json_obj(Doc, Options)) + send_json(Req, 200, Headers, couch_doc:to_json_obj(Doc, Options)) end. - +send_docs_multipart(Req, Results, Options) -> + OuterBoundary = couch_uuids:random(), + InnerBoundary = couch_uuids:random(), + CType = {"Content-Type", + "multipart/mixed; boundary=\"" ++ ?b2l(OuterBoundary) ++ "\""}, + {ok, Resp} = start_chunked_response(Req, 200, [CType]), + couch_httpd:send_chunk(Resp, <<"--", OuterBoundary/binary>>), + lists:foreach( + fun({ok, #doc{atts=Atts}=Doc}) -> + JsonBytes = ?JSON_ENCODE(couch_doc:to_json_obj(Doc, + [attachments,follows|Options])), + {ContentType, _Len} = couch_doc:len_doc_to_multi_part_stream( + InnerBoundary, JsonBytes, Atts, false), + couch_httpd:send_chunk(Resp, <<"\r\nContent-Type: ", + ContentType/binary, "\r\n\r\n">>), + couch_doc:doc_to_multi_part_stream(InnerBoundary, JsonBytes, Atts, + fun(Data) -> couch_httpd:send_chunk(Resp, Data) + end, false), + couch_httpd:send_chunk(Resp, <<"\r\n--", OuterBoundary/binary>>); + ({{not_found, missing}, RevId}) -> + RevStr = couch_doc:rev_to_str(RevId), + Json = ?JSON_ENCODE({[{"missing", RevStr}]}), + couch_httpd:send_chunk(Resp, + [<<"\r\nContent-Type: application/json; error=\"true\"\r\n\r\n">>, + Json, + <<"\r\n--", OuterBoundary/binary>>]) + end, Results), + couch_httpd:send_chunk(Resp, <<"--">>), + couch_httpd:last_chunk(Resp). receive_request_data(Req) -> {couch_httpd:recv(Req, 0), fun() -> receive_request_data(Req) end}. @@ -943,6 +827,8 @@ couch_doc_from_req(Req, DocId, #doc{revs=Revs}=Doc) -> case extract_header_rev(Req, ExplicitDocRev) of missing_rev -> Revs2 = {0, []}; + ExplicitDocRev -> + Revs2 = Revs; {Pos, Rev} -> Revs2 = {Pos, [Rev]} end, @@ -989,24 +875,66 @@ db_attachment_req(#httpd{method='GET'}=Req, Db, DocId, FileNameParts) -> case [A || A <- Atts, A#att.name == FileName] of [] -> throw({not_found, "Document is missing attachment"}); - [#att{type=Type, len=Len}=Att] -> + [#att{type=Type, encoding=Enc, disk_len=DiskLen, att_len=AttLen}=Att] -> Etag = couch_httpd:doc_etag(Doc), - couch_httpd:etag_respond(Req, Etag, fun() -> - {ok, Resp} = start_response_length(Req, 200, [ - {"ETag", Etag}, - {"Cache-Control", "must-revalidate"}, - {"Content-Type", binary_to_list(Type)} - ], integer_to_list(Len)), - couch_doc:att_foldl( - Att, - fun(BinSegment, _) -> send(Resp, BinSegment) end, - {ok, Resp} % Seed in case of 0 byte attachment. - ) - end) + ReqAcceptsAttEnc = lists:member( + atom_to_list(Enc), + couch_httpd:accepted_encodings(Req) + ), + Headers = [ + {"ETag", Etag}, + {"Cache-Control", "must-revalidate"}, + {"Content-Type", binary_to_list(Type)} + ] ++ case ReqAcceptsAttEnc of + true -> + [{"Content-Encoding", atom_to_list(Enc)}]; + _ -> + [] + end, + Len = case {Enc, ReqAcceptsAttEnc} of + {identity, _} -> + % stored and served in identity form + DiskLen; + {_, false} when DiskLen =/= AttLen -> + % Stored encoded, but client doesn't accept the encoding we used, + % so we need to decode on the fly. DiskLen is the identity length + % of the attachment. + DiskLen; + {_, true} -> + % Stored and served encoded. AttLen is the encoded length. + AttLen; + _ -> + % We received an encoded attachment and stored it as such, so we + % don't know the identity length. The client doesn't accept the + % encoding, and since we cannot serve a correct Content-Length + % header we'll fall back to a chunked response. + undefined + end, + AttFun = case ReqAcceptsAttEnc of + false -> + fun couch_doc:att_foldl_decode/3; + true -> + fun couch_doc:att_foldl/3 + end, + couch_httpd:etag_respond( + Req, + Etag, + fun() -> + case Len of + undefined -> + {ok, Resp} = start_chunked_response(Req, 200, Headers), + AttFun(Att, fun(Seg, _) -> send_chunk(Resp, Seg) end, ok), + last_chunk(Resp); + _ -> + {ok, Resp} = start_response_length(Req, 200, Headers, Len), + AttFun(Att, fun(Seg, _) -> send(Resp, Seg) end, ok) + end + end + ) end; -db_attachment_req(#httpd{method=Method}=Req, Db, DocId, FileNameParts) +db_attachment_req(#httpd{method=Method,mochi_req=MochiReq}=Req, Db, DocId, FileNameParts) when (Method == 'PUT') or (Method == 'DELETE') -> FileName = validate_attachment_name( mochiweb_util:join( @@ -1040,17 +968,44 @@ db_attachment_req(#httpd{method=Method}=Req, Db, DocId, FileNameParts) 0 -> <<"">>; Length when is_integer(Length) -> + Expect = case couch_httpd:header_value(Req, "expect") of + undefined -> + undefined; + Value when is_list(Value) -> + string:to_lower(Value) + end, + case Expect of + "100-continue" -> + MochiReq:start_raw_response({100, gb_trees:empty()}); + _Else -> + ok + end, + + fun() -> couch_httpd:recv(Req, 0) end; Length -> exit({length_not_integer, Length}) end, - len = case couch_httpd:header_value(Req,"Content-Length") of + att_len = case couch_httpd:header_value(Req,"Content-Length") of undefined -> undefined; Length -> list_to_integer(Length) end, - md5 = get_md5_header(Req) + md5 = get_md5_header(Req), + encoding = case string:to_lower(string:strip( + couch_httpd:header_value(Req,"Content-Encoding","identity") + )) of + "identity" -> + identity; + "gzip" -> + gzip; + _ -> + throw({ + bad_ctype, + "Only gzip and identity content-encodings are supported" + }) + end }] end, @@ -1076,7 +1031,8 @@ db_attachment_req(#httpd{method=Method}=Req, Db, DocId, FileNameParts) 'DELETE' -> {200, []}; _ -> - {201, [{"Location", absolute_uri(Req, "/" ++ + {201, [{"Etag", "\"" ++ ?b2l(couch_doc:rev_to_str(UpdatedRev)) ++ "\""}, + {"Location", absolute_uri(Req, "/" ++ binary_to_list(DbName) ++ "/" ++ binary_to_list(DocId) ++ "/" ++ binary_to_list(FileName) @@ -1112,17 +1068,6 @@ get_md5_header(Req) -> <<>> end. -parse_doc_format(FormatStr) when is_binary(FormatStr) -> - parse_doc_format(?b2l(FormatStr)); -parse_doc_format(FormatStr) when is_list(FormatStr) -> - SplitFormat = lists:splitwith(fun($/) -> false; (_) -> true end, FormatStr), - case SplitFormat of - {DesignName, [$/ | ShowName]} -> {?l2b(DesignName), ?l2b(ShowName)}; - _Else -> throw({bad_request, <<"Invalid doc format">>}) - end; -parse_doc_format(_BadFormatStr) -> - throw({bad_request, <<"Invalid doc format">>}). - parse_doc_query(Req) -> lists:foldl(fun({Key,Value}, Args) -> case {Key, Value} of @@ -1161,11 +1106,41 @@ parse_doc_query(Req) -> Args#doc_query_args{update_type=replicated_changes}; {"new_edits", "true"} -> Args#doc_query_args{update_type=interactive_edit}; + {"att_encoding_info", "true"} -> + Options = [att_encoding_info | Args#doc_query_args.options], + Args#doc_query_args{options=Options}; _Else -> % unknown key value pair, ignore. Args end end, #doc_query_args{}, couch_httpd:qs(Req)). +parse_changes_query(Req) -> + lists:foldl(fun({Key, Value}, Args) -> + case {Key, Value} of + {"feed", _} -> + Args#changes_args{feed=Value}; + {"descending", "true"} -> + Args#changes_args{dir=rev}; + {"since", _} -> + Args#changes_args{since=list_to_integer(Value)}; + {"limit", _} -> + Args#changes_args{limit=list_to_integer(Value)}; + {"style", _} -> + Args#changes_args{style=list_to_existing_atom(Value)}; + {"heartbeat", "true"} -> + Args#changes_args{heartbeat=true}; + {"heartbeat", _} -> + Args#changes_args{heartbeat=list_to_integer(Value)}; + {"timeout", _} -> + Args#changes_args{timeout=list_to_integer(Value)}; + {"include_docs", "true"} -> + Args#changes_args{include_docs=true}; + {"filter", _} -> + Args#changes_args{filter=Value}; + _Else -> % unknown key value pair, ignore. + Args + end + end, #changes_args{}, couch_httpd:qs(Req)). extract_header_rev(Req, ExplicitRev) when is_binary(ExplicitRev) or is_list(ExplicitRev)-> extract_header_rev(Req, couch_doc:parse_rev(ExplicitRev)); diff --git a/src/couchdb/couch_httpd_external.erl b/src/couchdb/couch_httpd_external.erl index efbe87b8..07202934 100644 --- a/src/couchdb/couch_httpd_external.erl +++ b/src/couchdb/couch_httpd_external.erl @@ -75,9 +75,10 @@ json_req_obj(#httpd{mochi_req=Req, % add headers... {[{<<"info">>, {Info}}, {<<"id">>, DocId}, + {<<"uuid">>, couch_uuids:new()}, {<<"method">>, Method}, {<<"path">>, Path}, - {<<"query">>, to_json_terms(Req:parse_qs())}, + {<<"query">>, json_query_keys(to_json_terms(Req:parse_qs()))}, {<<"headers">>, to_json_terms(Hlist)}, {<<"body">>, Body}, {<<"peer">>, ?l2b(Req:get(peer))}, @@ -95,6 +96,19 @@ to_json_terms([{Key, Value} | Rest], Acc) when is_atom(Key) -> to_json_terms([{Key, Value} | Rest], Acc) -> to_json_terms(Rest, [{list_to_binary(Key), list_to_binary(Value)} | Acc]). +json_query_keys({Json}) -> + json_query_keys(Json, []). +json_query_keys([], Acc) -> + {lists:reverse(Acc)}; +json_query_keys([{<<"startkey">>, Value} | Rest], Acc) -> + json_query_keys(Rest, [{<<"startkey">>, couch_util:json_decode(Value)}|Acc]); +json_query_keys([{<<"endkey">>, Value} | Rest], Acc) -> + json_query_keys(Rest, [{<<"endkey">>, couch_util:json_decode(Value)}|Acc]); +json_query_keys([{<<"key">>, Value} | Rest], Acc) -> + json_query_keys(Rest, [{<<"key">>, couch_util:json_decode(Value)}|Acc]); +json_query_keys([Term | Rest], Acc) -> + json_query_keys(Rest, [Term|Acc]). + send_external_response(#httpd{mochi_req=MochiReq}=Req, Response) -> #extern_resp_args{ code = Code, @@ -124,7 +138,7 @@ parse_external_response({Response}) -> Args#extern_resp_args{data=Value, ctype="text/html; charset=utf-8"}; {<<"base64">>, Value} -> Args#extern_resp_args{ - data=couch_util:decodeBase64(Value), + data=base64:decode(Value), ctype="application/binary" }; {<<"headers">>, {Headers}} -> diff --git a/src/couchdb/couch_httpd_misc_handlers.erl b/src/couchdb/couch_httpd_misc_handlers.erl index 2d67b321..db1b2ca1 100644 --- a/src/couchdb/couch_httpd_misc_handlers.erl +++ b/src/couchdb/couch_httpd_misc_handlers.erl @@ -15,7 +15,7 @@ -export([handle_welcome_req/2,handle_favicon_req/2,handle_utils_dir_req/2, handle_all_dbs_req/1,handle_replicate_req/1,handle_restart_req/1, handle_uuids_req/1,handle_config_req/1,handle_log_req/1, - handle_task_status_req/1,handle_sleep_req/1]). + handle_task_status_req/1]). -export([increment_update_seq_req/2]). @@ -46,6 +46,7 @@ handle_favicon_req(#httpd{method='GET'}=Req, DocumentRoot) -> {"Expires", httpd_util:rfc1123_date(OneYearFromNow)} ], couch_httpd:serve_file(Req, "favicon.ico", DocumentRoot, CachingHeaders); + handle_favicon_req(Req, _) -> send_method_not_allowed(Req, "GET,HEAD"). @@ -63,13 +64,6 @@ handle_utils_dir_req(#httpd{method='GET'}=Req, DocumentRoot) -> handle_utils_dir_req(Req, _) -> send_method_not_allowed(Req, "GET,HEAD"). -handle_sleep_req(#httpd{method='GET'}=Req) -> - Time = list_to_integer(couch_httpd:qs_value(Req, "time")), - receive snicklefart -> ok after Time -> ok end, - send_json(Req, {[{ok, true}]}); -handle_sleep_req(Req) -> - send_method_not_allowed(Req, "GET,HEAD"). - handle_all_dbs_req(#httpd{method='GET'}=Req) -> {ok, DbNames} = couch_server:all_databases(), send_json(Req, DbNames); @@ -85,25 +79,38 @@ handle_task_status_req(Req) -> send_method_not_allowed(Req, "GET,HEAD"). handle_replicate_req(#httpd{method='POST'}=Req) -> + couch_httpd:validate_ctype(Req, "application/json"), PostBody = couch_httpd:json_body_obj(Req), try couch_rep:replicate(PostBody, Req#httpd.user_ctx) of {ok, {continuous, RepId}} -> send_json(Req, 202, {[{ok, true}, {<<"_local_id">>, RepId}]}); + {ok, {cancelled, RepId}} -> + send_json(Req, 200, {[{ok, true}, {<<"_local_id">>, RepId}]}); {ok, {JsonResults}} -> send_json(Req, {[{ok, true} | JsonResults]}); {error, {Type, Details}} -> send_json(Req, 500, {[{error, Type}, {reason, Details}]}); + {error, not_found} -> + send_json(Req, 404, {[{error, not_found}]}); {error, Reason} -> - send_json(Req, 500, {[{error, Reason}]}) + try + send_json(Req, 500, {[{error, Reason}]}) + catch + exit:{json_encode, _} -> + send_json(Req, 500, {[{error, couch_util:to_binary(Reason)}]}) + end catch throw:{db_not_found, Msg} -> - send_json(Req, 404, {[{error, db_not_found}, {reason, Msg}]}) + send_json(Req, 404, {[{error, db_not_found}, {reason, Msg}]}); + throw:{unauthorized, Msg} -> + send_json(Req, 404, {[{error, unauthorized}, {reason, Msg}]}) end; handle_replicate_req(Req) -> send_method_not_allowed(Req, "POST"). handle_restart_req(#httpd{method='POST'}=Req) -> + couch_httpd:validate_ctype(Req, "application/json"), ok = couch_httpd:verify_is_server_admin(Req), couch_server_sup:restart_core_server(), send_json(Req, 200, {[{ok, true}]}); @@ -191,6 +198,7 @@ handle_config_req(Req) -> % httpd db handlers increment_update_seq_req(#httpd{method='POST'}=Req, Db) -> + couch_httpd:validate_ctype(Req, "application/json"), {ok, NewSeq} = couch_db:increment_update_seq(Db), send_json(Req, {[{ok, true}, {update_seq, NewSeq} @@ -201,6 +209,7 @@ increment_update_seq_req(Req, _Db) -> % httpd log handlers handle_log_req(#httpd{method='GET'}=Req) -> + ok = couch_httpd:verify_is_server_admin(Req), Bytes = list_to_integer(couch_httpd:qs_value(Req, "bytes", "1000")), Offset = list_to_integer(couch_httpd:qs_value(Req, "offset", "0")), Chunk = couch_log:read(Bytes, Offset), diff --git a/src/couchdb/couch_httpd_oauth.erl b/src/couchdb/couch_httpd_oauth.erl index ddf84008..05ee10e2 100644 --- a/src/couchdb/couch_httpd_oauth.erl +++ b/src/couchdb/couch_httpd_oauth.erl @@ -18,9 +18,9 @@ % OAuth auth handler using per-node user db oauth_authentication_handler(#httpd{mochi_req=MochiReq}=Req) -> serve_oauth(Req, fun(URL, Params, Consumer, Signature) -> - AccessToken = proplists:get_value("oauth_token", Params), + AccessToken = couch_util:get_value("oauth_token", Params), case couch_config:get("oauth_token_secrets", AccessToken) of - undefined -> + undefined -> couch_httpd:send_error(Req, 400, <<"invalid_token">>, <<"Invalid OAuth token.">>); TokenSecret -> @@ -41,17 +41,17 @@ set_user_ctx(Req, AccessToken) -> undefined -> throw({bad_request, unknown_oauth_token}); Value -> ?l2b(Value) end, - case couch_httpd_auth:get_user(Name) of + case couch_auth_cache:get_user_creds(Name) of nil -> Req; User -> - Roles = proplists:get_value(<<"roles">>, User, []), + Roles = couch_util:get_value(<<"roles">>, User, []), Req#httpd{user_ctx=#user_ctx{name=Name, roles=Roles}} end. % OAuth request_token handle_oauth_req(#httpd{path_parts=[_OAuth, <<"request_token">>], method=Method}=Req) -> serve_oauth(Req, fun(URL, Params, Consumer, Signature) -> - AccessToken = proplists:get_value("oauth_token", Params), + AccessToken = couch_util:get_value("oauth_token", Params), TokenSecret = couch_config:get("oauth_token_secrets", AccessToken), case oauth:verify(Signature, atom_to_list(Method), URL, Params, Consumer, TokenSecret) of true -> @@ -88,7 +88,7 @@ serve_oauth_authorize(#httpd{method=Method}=Req) -> 'GET' -> % Confirm with the User that they want to authenticate the Consumer serve_oauth(Req, fun(URL, Params, Consumer, Signature) -> - AccessToken = proplists:get_value("oauth_token", Params), + AccessToken = couch_util:get_value("oauth_token", Params), TokenSecret = couch_config:get("oauth_token_secrets", AccessToken), case oauth:verify(Signature, "GET", URL, Params, Consumer, TokenSecret) of true -> @@ -100,7 +100,7 @@ serve_oauth_authorize(#httpd{method=Method}=Req) -> 'POST' -> % If the User has confirmed, we direct the User back to the Consumer with a verification code serve_oauth(Req, fun(URL, Params, Consumer, Signature) -> - AccessToken = proplists:get_value("oauth_token", Params), + AccessToken = couch_util:get_value("oauth_token", Params), TokenSecret = couch_config:get("oauth_token_secrets", AccessToken), case oauth:verify(Signature, "POST", URL, Params, Consumer, TokenSecret) of true -> @@ -129,24 +129,24 @@ serve_oauth(#httpd{mochi_req=MochiReq}=Req, Fun, FailSilently) -> end end, HeaderParams = oauth_uri:params_from_header_string(AuthHeader), - %Realm = proplists:get_value("realm", HeaderParams), + %Realm = couch_util:get_value("realm", HeaderParams), Params = proplists:delete("realm", HeaderParams) ++ MochiReq:parse_qs(), ?LOG_DEBUG("OAuth Params: ~p", [Params]), - case proplists:get_value("oauth_version", Params, "1.0") of + case couch_util:get_value("oauth_version", Params, "1.0") of "1.0" -> - case proplists:get_value("oauth_consumer_key", Params, undefined) of + case couch_util:get_value("oauth_consumer_key", Params, undefined) of undefined -> case FailSilently of true -> Req; false -> couch_httpd:send_error(Req, 400, <<"invalid_consumer">>, <<"Invalid consumer.">>) end; ConsumerKey -> - SigMethod = proplists:get_value("oauth_signature_method", Params), + SigMethod = couch_util:get_value("oauth_signature_method", Params), case consumer_lookup(ConsumerKey, SigMethod) of none -> couch_httpd:send_error(Req, 400, <<"invalid_consumer">>, <<"Invalid consumer (key or signature method).">>); Consumer -> - Signature = proplists:get_value("oauth_signature", Params), + Signature = couch_util:get_value("oauth_signature", Params), URL = couch_httpd:absolute_uri(Req, MochiReq:get(raw_path)), Fun(URL, proplists:delete("oauth_signature", Params), Consumer, Signature) diff --git a/src/couchdb/couch_httpd_rewrite.erl b/src/couchdb/couch_httpd_rewrite.erl new file mode 100644 index 00000000..6c3d0e3c --- /dev/null +++ b/src/couchdb/couch_httpd_rewrite.erl @@ -0,0 +1,428 @@ +% 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. +% +% bind_path is based on bind method from Webmachine + + +%% @doc Module for URL rewriting by pattern matching. + +-module(couch_httpd_rewrite). +-export([handle_rewrite_req/3]). +-include("couch_db.hrl"). + +-define(SEPARATOR, $\/). +-define(MATCH_ALL, {bind, <<"*">>}). + + +%% doc The http rewrite handler. All rewriting is done from +%% /dbname/_design/ddocname/_rewrite by default. +%% +%% each rules should be in rewrites member of the design doc. +%% Ex of a complete rule : +%% +%% { +%% .... +%% "rewrites": [ +%% { +%% "from": "", +%% "to": "index.html", +%% "method": "GET", +%% "query": {} +%% } +%% ] +%% } +%% +%% from: is the path rule used to bind current uri to the rule. It +%% use pattern matching for that. +%% +%% to: rule to rewrite an url. It can contain variables depending on binding +%% variables discovered during pattern matching and query args (url args and from +%% the query member.) +%% +%% method: method to bind the request method to the rule. by default "*" +%% query: query args you want to define they can contain dynamic variable +%% by binding the key to the bindings +%% +%% +%% to and from are path with patterns. pattern can be string starting with ":" or +%% "*". ex: +%% /somepath/:var/* +%% +%% This path is converted in erlang list by splitting "/". Each var are +%% converted in atom. "*" is converted to '*' atom. The pattern matching is done +%% by splitting "/" in request url in a list of token. A string pattern will +%% match equal token. The star atom ('*' in single quotes) will match any number +%% of tokens, but may only be present as the last pathtern in a pathspec. If all +%% tokens are matched and all pathterms are used, then the pathspec matches. It works +%% like webmachine. Each identified token will be reused in to rule and in query +%% +%% The pattern matching is done by first matching the request method to a rule. by +%% default all methods match a rule. (method is equal to "*" by default). Then +%% It will try to match the path to one rule. If no rule match, then a 404 error +%% is displayed. +%% +%% Once a rule is found we rewrite the request url using the "to" and +%% "query" members. The identified token are matched to the rule and +%% will replace var. if '*' is found in the rule it will contain the remaining +%% part if it exists. +%% +%% Examples: +%% +%% Dispatch rule URL TO Tokens +%% +%% {"from": "/a/b", /a/b?k=v /some/b?k=v var =:= b +%% "to": "/some/"} k = v +%% +%% {"from": "/a/b", /a/b /some/b?var=b var =:= b +%% "to": "/some/:var"} +%% +%% {"from": "/a", /a /some +%% "to": "/some/*"} +%% +%% {"from": "/a/*", /a/b/c /some/b/c +%% "to": "/some/*"} +%% +%% {"from": "/a", /a /some +%% "to": "/some/*"} +%% +%% {"from": "/a/:foo/*", /a/b/c /some/b/c?foo=b foo =:= b +%% "to": "/some/:foo/*"} +%% +%% {"from": "/a/:foo", /a/b /some/?k=b&foo=b foo =:= b +%% "to": "/some", +%% "query": { +%% "k": ":foo" +%% }} +%% +%% {"from": "/a", /a?foo=b /some/b foo =:= b +%% "to": "/some/:foo", +%% }} + + + +handle_rewrite_req(#httpd{ + path_parts=[DbName, <<"_design">>, DesignName, _Rewrite|PathParts], + method=Method, + mochi_req=MochiReq}=Req, _Db, DDoc) -> + + % we are in a design handler + DesignId = <<"_design/", DesignName/binary>>, + Prefix = <<"/", DbName/binary, "/", DesignId/binary>>, + QueryList = couch_httpd:qs(Req), + QueryList1 = [{to_binding(K), V} || {K, V} <- QueryList], + + #doc{body={Props}} = DDoc, + + % get rules from ddoc + case couch_util:get_value(<<"rewrites">>, Props) of + undefined -> + couch_httpd:send_error(Req, 404, <<"rewrite_error">>, + <<"Invalid path.">>); + Bin when is_binary(Bin) -> + couch_httpd:send_error(Req, 400, <<"rewrite_error">>, + <<"Rewrite rules are a String. They must be a JSON Array.">>); + Rules -> + % create dispatch list from rules + DispatchList = [make_rule(Rule) || {Rule} <- Rules], + + %% get raw path by matching url to a rule. + RawPath = case try_bind_path(DispatchList, couch_util:to_binary(Method), PathParts, + QueryList1) of + no_dispatch_path -> + throw(not_found); + {NewPathParts, Bindings} -> + Parts = [quote_plus(X) || X <- NewPathParts], + + % build new path, reencode query args, eventually convert + % them to json + Path = lists:append( + string:join(Parts, [?SEPARATOR]), + case Bindings of + [] -> []; + _ -> [$?, encode_query(Bindings)] + end), + + % if path is relative detect it and rewrite path + case mochiweb_util:safe_relative_path(Path) of + undefined -> + ?b2l(Prefix) ++ "/" ++ Path; + P1 -> + ?b2l(Prefix) ++ "/" ++ P1 + end + + end, + + % normalize final path (fix levels "." and "..") + RawPath1 = ?b2l(iolist_to_binary(normalize_path(RawPath))), + + ?LOG_DEBUG("rewrite to ~p ~n", [RawPath1]), + + % build a new mochiweb request + MochiReq1 = mochiweb_request:new(MochiReq:get(socket), + MochiReq:get(method), + RawPath1, + MochiReq:get(version), + MochiReq:get(headers)), + + % cleanup, It force mochiweb to reparse raw uri. + MochiReq1:cleanup(), + + #httpd{ + db_url_handlers = DbUrlHandlers, + design_url_handlers = DesignUrlHandlers, + default_fun = DefaultFun, + url_handlers = UrlHandlers + } = Req, + couch_httpd:handle_request_int(MochiReq1, DefaultFun, + UrlHandlers, DbUrlHandlers, DesignUrlHandlers) + end. + +quote_plus({bind, X}) -> + mochiweb_util:quote_plus(X); +quote_plus(X) -> + mochiweb_util:quote_plus(X). + +%% @doc Try to find a rule matching current url. If none is found +%% 404 error not_found is raised +try_bind_path([], _Method, _PathParts, _QueryList) -> + no_dispatch_path; +try_bind_path([Dispatch|Rest], Method, PathParts, QueryList) -> + [{PathParts1, Method1}, RedirectPath, QueryArgs] = Dispatch, + case bind_method(Method1, Method) of + true -> + case bind_path(PathParts1, PathParts, []) of + {ok, Remaining, Bindings} -> + Bindings1 = Bindings ++ QueryList, + % we parse query args from the rule and fill + % it eventually with bindings vars + QueryArgs1 = make_query_list(QueryArgs, Bindings1, []), + % remove params in QueryLists1 that are already in + % QueryArgs1 + Bindings2 = lists:foldl(fun({K, V}, Acc) -> + K1 = to_binding(K), + KV = case couch_util:get_value(K1, QueryArgs1) of + undefined -> [{K1, V}]; + _V1 -> [] + end, + Acc ++ KV + end, [], Bindings1), + + FinalBindings = Bindings2 ++ QueryArgs1, + NewPathParts = make_new_path(RedirectPath, FinalBindings, + Remaining, []), + {NewPathParts, FinalBindings}; + fail -> + try_bind_path(Rest, Method, PathParts, QueryList) + end; + false -> + try_bind_path(Rest, Method, PathParts, QueryList) + end. + +%% rewriting dynamically the quey list given as query member in +%% rewrites. Each value is replaced by one binding or an argument +%% passed in url. +make_query_list([], _Bindings, Acc) -> + Acc; +make_query_list([{Key, {Value}}|Rest], Bindings, Acc) -> + Value1 = to_json({Value}), + make_query_list(Rest, Bindings, [{to_binding(Key), Value1}|Acc]); +make_query_list([{Key, Value}|Rest], Bindings, Acc) when is_binary(Value) -> + Value1 = replace_var(Key, Value, Bindings), + make_query_list(Rest, Bindings, [{to_binding(Key), Value1}|Acc]); +make_query_list([{Key, Value}|Rest], Bindings, Acc) when is_list(Value) -> + Value1 = replace_var(Key, Value, Bindings), + make_query_list(Rest, Bindings, [{to_binding(Key), Value1}|Acc]); +make_query_list([{Key, Value}|Rest], Bindings, Acc) -> + make_query_list(Rest, Bindings, [{to_binding(Key), Value}|Acc]). + +replace_var(Key, Value, Bindings) -> + case Value of + <<":", Var/binary>> -> + get_var(Var, Bindings, Value); + _ when is_list(Value) -> + Value1 = lists:foldr(fun(V, Acc) -> + V1 = case V of + <<":", VName/binary>> -> + case get_var(VName, Bindings, V) of + V2 when is_list(V2) -> + iolist_to_binary(V2); + V2 -> V2 + end; + _ -> + + V + end, + [V1|Acc] + end, [], Value), + to_json(Value1); + _ when is_binary(Value) -> + Value; + _ -> + case Key of + <<"key">> -> to_json(Value); + <<"startkey">> -> to_json(Value); + <<"endkey">> -> to_json(Value); + _ -> + lists:flatten(?JSON_ENCODE(Value)) + end + end. + + +get_var(VarName, Props, Default) -> + VarName1 = to_binding(VarName), + couch_util:get_value(VarName1, Props, Default). + +%% doc: build new patch from bindings. bindings are query args +%% (+ dynamic query rewritten if needed) and bindings found in +%% bind_path step. +make_new_path([], _Bindings, _Remaining, Acc) -> + lists:reverse(Acc); +make_new_path([?MATCH_ALL], _Bindings, Remaining, Acc) -> + Acc1 = lists:reverse(Acc) ++ Remaining, + Acc1; +make_new_path([?MATCH_ALL|_Rest], _Bindings, Remaining, Acc) -> + Acc1 = lists:reverse(Acc) ++ Remaining, + Acc1; +make_new_path([{bind, P}|Rest], Bindings, Remaining, Acc) -> + P2 = case couch_util:get_value({bind, P}, Bindings) of + undefined -> << "undefined">>; + P1 -> P1 + end, + make_new_path(Rest, Bindings, Remaining, [P2|Acc]); +make_new_path([P|Rest], Bindings, Remaining, Acc) -> + make_new_path(Rest, Bindings, Remaining, [P|Acc]). + + +%% @doc If method of the query fith the rule method. If the +%% method rule is '*', which is the default, all +%% request method will bind. It allows us to make rules +%% depending on HTTP method. +bind_method(?MATCH_ALL, _Method) -> + true; +bind_method({bind, Method}, Method) -> + true; +bind_method(_, _) -> + false. + + +%% @doc bind path. Using the rule from we try to bind variables given +%% to the current url by pattern matching +bind_path([], [], Bindings) -> + {ok, [], Bindings}; +bind_path([?MATCH_ALL], Rest, Bindings) when is_list(Rest) -> + {ok, Rest, Bindings}; +bind_path(_, [], _) -> + fail; +bind_path([{bind, Token}|RestToken],[Match|RestMatch],Bindings) -> + bind_path(RestToken, RestMatch, [{{bind, Token}, Match}|Bindings]); +bind_path([Token|RestToken], [Token|RestMatch], Bindings) -> + bind_path(RestToken, RestMatch, Bindings); +bind_path(_, _, _) -> + fail. + + +%% normalize path. +normalize_path(Path) -> + "/" ++ string:join(normalize_path1(string:tokens(Path, + "/"), []), [?SEPARATOR]). + + +normalize_path1([], Acc) -> + lists:reverse(Acc); +normalize_path1([".."|Rest], Acc) -> + Acc1 = case Acc of + [] -> [".."|Acc]; + [T|_] when T =:= ".." -> [".."|Acc]; + [_|R] -> R + end, + normalize_path1(Rest, Acc1); +normalize_path1(["."|Rest], Acc) -> + normalize_path1(Rest, Acc); +normalize_path1([Path|Rest], Acc) -> + normalize_path1(Rest, [Path|Acc]). + + +%% @doc transform json rule in erlang for pattern matching +make_rule(Rule) -> + Method = case couch_util:get_value(<<"method">>, Rule) of + undefined -> ?MATCH_ALL; + M -> to_binding(M) + end, + QueryArgs = case couch_util:get_value(<<"query">>, Rule) of + undefined -> []; + {Args} -> Args + end, + FromParts = case couch_util:get_value(<<"from">>, Rule) of + undefined -> [?MATCH_ALL]; + From -> + parse_path(From) + end, + ToParts = case couch_util:get_value(<<"to">>, Rule) of + undefined -> + throw({error, invalid_rewrite_target}); + To -> + parse_path(To) + end, + [{FromParts, Method}, ToParts, QueryArgs]. + +parse_path(Path) -> + {ok, SlashRE} = re:compile(<<"\\/">>), + path_to_list(re:split(Path, SlashRE), [], 0). + +%% @doc convert a path rule (from or to) to an erlang list +%% * and path variable starting by ":" are converted +%% in erlang atom. +path_to_list([], Acc, _DotDotCount) -> + lists:reverse(Acc); +path_to_list([<<>>|R], Acc, DotDotCount) -> + path_to_list(R, Acc, DotDotCount); +path_to_list([<<"*">>|R], Acc, DotDotCount) -> + path_to_list(R, [?MATCH_ALL|Acc], DotDotCount); +path_to_list([<<"..">>|R], Acc, DotDotCount) when DotDotCount == 2 -> + case couch_config:get("httpd", "secure_rewrites", "true") of + "false" -> + path_to_list(R, [<<"..">>|Acc], DotDotCount+1); + _Else -> + ?LOG_INFO("insecure_rewrite_rule ~p blocked", [lists:reverse(Acc) ++ [<<"..">>] ++ R]), + throw({insecure_rewrite_rule, "too many ../.. segments"}) + end; +path_to_list([<<"..">>|R], Acc, DotDotCount) -> + path_to_list(R, [<<"..">>|Acc], DotDotCount+1); +path_to_list([P|R], Acc, DotDotCount) -> + P1 = case P of + <<":", Var/binary>> -> + to_binding(Var); + _ -> P + end, + path_to_list(R, [P1|Acc], DotDotCount). + +encode_query(Props) -> + Props1 = lists:foldl(fun ({{bind, K}, V}, Acc) -> + V1 = case is_list(V) orelse is_binary(V) of + true -> V; + false -> + % probably it's a number + quote_plus(V) + end, + [{K, V1} | Acc] + end, [], Props), + lists:flatten(mochiweb_util:urlencode(Props1)). + +to_binding({bind, V}) -> + {bind, V}; +to_binding(V) when is_list(V) -> + to_binding(?l2b(V)); +to_binding(V) -> + {bind, V}. + +to_json(V) -> + iolist_to_binary(?JSON_ENCODE(V)). diff --git a/src/couchdb/couch_httpd_show.erl b/src/couchdb/couch_httpd_show.erl index 467c0a42..d50ca83a 100644 --- a/src/couchdb/couch_httpd_show.erl +++ b/src/couchdb/couch_httpd_show.erl @@ -13,7 +13,7 @@ -module(couch_httpd_show). -export([handle_doc_show_req/3, handle_doc_update_req/3, handle_view_list_req/3, - handle_doc_show/5, handle_view_list/6, get_fun_key/3]). + handle_view_list/6, get_fun_key/3]). -include("couch_db.hrl"). @@ -22,18 +22,42 @@ start_json_response/2,send_chunk/2,last_chunk/1,send_chunked_error/2, start_chunked_response/3, send_error/4]). -% /db/_design/foo/show/bar/docid -% show converts a json doc to a response of any content-type. + +% /db/_design/foo/_show/bar/docid +% show converts a json doc to a response of any content-type. % it looks up the doc an then passes it to the query server. % then it sends the response from the query server to the http client. + +maybe_open_doc(Db, DocId) -> + case catch couch_httpd_db:couch_doc_open(Db, DocId, nil, [conflicts]) of + {not_found, missing} -> nil; + {not_found,deleted} -> nil; + Doc -> Doc + end. handle_doc_show_req(#httpd{ path_parts=[_, _, _, _, ShowName, DocId] }=Req, Db, DDoc) -> + % open the doc - Doc = couch_httpd_db:couch_doc_open(Db, DocId, nil, [conflicts]), + Doc = maybe_open_doc(Db, DocId), + % we don't handle revs here b/c they are an internal api % returns 404 if there is no doc with DocId - handle_doc_show(Req, Db, DDoc, ShowName, Doc); + handle_doc_show(Req, Db, DDoc, ShowName, Doc, DocId); + +handle_doc_show_req(#httpd{ + path_parts=[_, _, _, _, ShowName, DocId|Rest] + }=Req, Db, DDoc) -> + + DocParts = [DocId|Rest], + DocId1 = ?l2b(string:join([?b2l(P)|| P <- DocParts], "/")), + + % open the doc + Doc = maybe_open_doc(Db, DocId1), + + % we don't handle revs here b/c they are an internal api + % pass 404 docs to the show function + handle_doc_show(Req, Db, DDoc, ShowName, Doc, DocId1); handle_doc_show_req(#httpd{ path_parts=[_, _, _, _, ShowName] @@ -45,12 +69,15 @@ handle_doc_show_req(Req, _Db, _DDoc) -> send_error(Req, 404, <<"show_error">>, <<"Invalid path.">>). handle_doc_show(Req, Db, DDoc, ShowName, Doc) -> + handle_doc_show(Req, Db, DDoc, ShowName, Doc, null). + +handle_doc_show(Req, Db, DDoc, ShowName, Doc, DocId) -> % get responder for ddoc/showname CurrentEtag = show_etag(Req, Doc, DDoc, []), couch_httpd:etag_respond(Req, CurrentEtag, fun() -> - JsonReq = couch_httpd_external:json_req_obj(Req, Db), + JsonReq = couch_httpd_external:json_req_obj(Req, Db, DocId), JsonDoc = couch_query_servers:json_doc(Doc), - [<<"resp">>, ExternalResp] = + [<<"resp">>, ExternalResp] = couch_query_servers:ddoc_prompt(DDoc, [<<"shows">>, ShowName], [JsonDoc, JsonReq]), JsonResp = apply_etag(ExternalResp, CurrentEtag), couch_httpd_external:send_external_response(Req, JsonResp) @@ -68,7 +95,7 @@ show_etag(#httpd{user_ctx=UserCtx}=Req, Doc, DDoc, More) -> get_fun_key(DDoc, Type, Name) -> #doc{body={Props}} = DDoc, - Lang = proplists:get_value(<<"language">>, Props, <<"javascript">>), + Lang = couch_util:get_value(<<"language">>, Props, <<"javascript">>), Src = couch_util:get_nested_json_value({Props}, [Type, Name]), {Lang, Src}. @@ -98,22 +125,27 @@ handle_doc_update_req(Req, _Db, _DDoc) -> send_doc_update_response(Req, Db, DDoc, UpdateName, Doc, DocId) -> JsonReq = couch_httpd_external:json_req_obj(Req, Db, DocId), JsonDoc = couch_query_servers:json_doc(Doc), - case couch_query_servers:ddoc_prompt(DDoc, [<<"updates">>, UpdateName], [JsonDoc, JsonReq]) of - [<<"up">>, {NewJsonDoc}, JsonResp] -> - Options = case couch_httpd:header_value(Req, "X-Couch-Full-Commit", "false") of + {Code, JsonResp1} = case couch_query_servers:ddoc_prompt(DDoc, + [<<"updates">>, UpdateName], [JsonDoc, JsonReq]) of + [<<"up">>, {NewJsonDoc}, {JsonResp}] -> + Options = case couch_httpd:header_value(Req, "X-Couch-Full-Commit", + "false") of "true" -> [full_commit]; _ -> [] end, NewDoc = couch_doc:from_json_obj({NewJsonDoc}), - Code = 201, - {ok, _NewRev} = couch_db:update_doc(Db, NewDoc, Options); + {ok, NewRev} = couch_db:update_doc(Db, NewDoc, Options), + NewRevStr = couch_doc:rev_to_str(NewRev), + JsonRespWithRev = {[{<<"headers">>, + {[{<<"X-Couch-Update-NewRev">>, NewRevStr}]}} | JsonResp]}, + {201, JsonRespWithRev}; [<<"up">>, _Other, JsonResp] -> - Code = 200, - ok + {200, JsonResp} end, - JsonResp2 = json_apply_field({<<"code">>, Code}, JsonResp), + + JsonResp2 = couch_util:json_apply_field({<<"code">>, Code}, JsonResp1), % todo set location field couch_httpd_external:send_external_response(Req, JsonResp2). @@ -136,7 +168,7 @@ handle_view_list_req(#httpd{method='POST', % {Props2} = couch_httpd:json_body(Req), ReqBody = couch_httpd:body(Req), {Props2} = ?JSON_DECODE(ReqBody), - Keys = proplists:get_value(<<"keys">>, Props2, nil), + Keys = couch_util:get_value(<<"keys">>, Props2, nil), handle_view_list(Req#httpd{req_body=ReqBody}, Db, DDoc, ListName, {DesignName, ViewName}, Keys); handle_view_list_req(#httpd{method='POST', @@ -144,7 +176,7 @@ handle_view_list_req(#httpd{method='POST', % {Props2} = couch_httpd:json_body(Req), ReqBody = couch_httpd:body(Req), {Props2} = ?JSON_DECODE(ReqBody), - Keys = proplists:get_value(<<"keys">>, Props2, nil), + Keys = couch_util:get_value(<<"keys">>, Props2, nil), handle_view_list(Req#httpd{req_body=ReqBody}, Db, DDoc, ListName, {ViewDesignName, ViewName}, Keys); handle_view_list_req(#httpd{method='POST'}=Req, _Db, _DDoc) -> @@ -156,23 +188,23 @@ handle_view_list_req(Req, _Db, _DDoc) -> handle_view_list(Req, Db, DDoc, LName, {ViewDesignName, ViewName}, Keys) -> ViewDesignId = <<"_design/", ViewDesignName/binary>>, {ViewType, View, Group, QueryArgs} = couch_httpd_view:load_view(Req, Db, {ViewDesignId, ViewName}, Keys), - Etag = list_etag(Req, Db, Group, {couch_httpd:doc_etag(DDoc), Keys}), + Etag = list_etag(Req, Db, Group, {couch_httpd:doc_etag(DDoc), Keys}), couch_httpd:etag_respond(Req, Etag, fun() -> - output_list(ViewType, Req, Db, DDoc, LName, View, QueryArgs, Etag, Keys) - end). + output_list(ViewType, Req, Db, DDoc, LName, View, QueryArgs, Etag, Keys, Group) + end). list_etag(#httpd{user_ctx=UserCtx}=Req, Db, Group, More) -> Accept = couch_httpd:header_value(Req, "Accept"), couch_httpd_view:view_group_etag(Group, Db, {More, Accept, UserCtx#user_ctx.roles}). -output_list(map, Req, Db, DDoc, LName, View, QueryArgs, Etag, Keys) -> - output_map_list(Req, Db, DDoc, LName, View, QueryArgs, Etag, Keys); -output_list(reduce, Req, Db, DDoc, LName, View, QueryArgs, Etag, Keys) -> - output_reduce_list(Req, Db, DDoc, LName, View, QueryArgs, Etag, Keys). +output_list(map, Req, Db, DDoc, LName, View, QueryArgs, Etag, Keys, Group) -> + output_map_list(Req, Db, DDoc, LName, View, QueryArgs, Etag, Keys, Group); +output_list(reduce, Req, Db, DDoc, LName, View, QueryArgs, Etag, Keys, Group) -> + output_reduce_list(Req, Db, DDoc, LName, View, QueryArgs, Etag, Keys, Group). % next step: % use with_ddoc_proc/2 to make this simpler -output_map_list(Req, Db, DDoc, LName, View, QueryArgs, Etag, Keys) -> +output_map_list(Req, Db, DDoc, LName, View, QueryArgs, Etag, Keys, Group) -> #view_query_args{ limit = Limit, skip = SkipCount @@ -188,12 +220,13 @@ output_map_list(Req, Db, DDoc, LName, View, QueryArgs, Etag, Keys) -> reduce_count = fun couch_view:reduce_to_count/1, start_response = StartListRespFun = make_map_start_resp_fun(QServer, Db, LName), send_row = make_map_send_row_fun(QServer) - }, + }, + CurrentSeq = Group#group.current_seq, {ok, _, FoldResult} = case Keys of nil -> - FoldlFun = couch_httpd_view:make_view_fold_fun(Req, QueryArgs, Etag, Db, RowCount, ListFoldHelpers), - couch_view:fold(View, FoldlFun, FoldAccInit, + FoldlFun = couch_httpd_view:make_view_fold_fun(Req, QueryArgs, Etag, Db, CurrentSeq, RowCount, ListFoldHelpers), + couch_view:fold(View, FoldlFun, FoldAccInit, couch_httpd_view:make_key_options(QueryArgs)); Keys -> lists:foldl( @@ -202,27 +235,29 @@ output_map_list(Req, Db, DDoc, LName, View, QueryArgs, Etag, Keys) -> start_key = Key, end_key = Key }, - FoldlFun = couch_httpd_view:make_view_fold_fun(Req, QueryArgs2, Etag, Db, RowCount, ListFoldHelpers), + FoldlFun = couch_httpd_view:make_view_fold_fun(Req, QueryArgs2, Etag, Db, CurrentSeq, RowCount, ListFoldHelpers), couch_view:fold(View, FoldlFun, FoldAcc, couch_httpd_view:make_key_options(QueryArgs2)) end, {ok, nil, FoldAccInit}, Keys) end, - finish_list(Req, QServer, Etag, FoldResult, StartListRespFun, RowCount) + finish_list(Req, QServer, Etag, FoldResult, StartListRespFun, CurrentSeq, RowCount) end). -output_reduce_list(Req, Db, DDoc, LName, View, QueryArgs, Etag, Keys) -> +output_reduce_list(Req, Db, DDoc, LName, View, QueryArgs, Etag, Keys, Group) -> #view_query_args{ limit = Limit, skip = SkipCount, group_level = GroupLevel } = QueryArgs, + CurrentSeq = Group#group.current_seq, + couch_query_servers:with_ddoc_proc(DDoc, fun(QServer) -> StartListRespFun = make_reduce_start_resp_fun(QServer, Db, LName), SendListRowFun = make_reduce_send_row_fun(QServer, Db), {ok, GroupRowsFun, RespFun} = couch_httpd_view:make_reduce_fold_funs(Req, - GroupLevel, QueryArgs, Etag, + GroupLevel, QueryArgs, Etag, CurrentSeq, #reduce_fold_helper_funs{ start_response = StartListRespFun, send_row = SendListRowFun @@ -230,36 +265,36 @@ output_reduce_list(Req, Db, DDoc, LName, View, QueryArgs, Etag, Keys) -> FoldAccInit = {Limit, SkipCount, undefined, []}, {ok, FoldResult} = case Keys of nil -> - couch_view:fold_reduce(View, RespFun, FoldAccInit, [{key_group_fun, GroupRowsFun} | + couch_view:fold_reduce(View, RespFun, FoldAccInit, [{key_group_fun, GroupRowsFun} | couch_httpd_view:make_key_options(QueryArgs)]); Keys -> lists:foldl( fun(Key, {ok, FoldAcc}) -> couch_view:fold_reduce(View, RespFun, FoldAcc, - [{key_group_fun, GroupRowsFun} | + [{key_group_fun, GroupRowsFun} | couch_httpd_view:make_key_options( QueryArgs#view_query_args{start_key=Key, end_key=Key})] - ) + ) end, {ok, FoldAccInit}, Keys) end, - finish_list(Req, QServer, Etag, FoldResult, StartListRespFun, null) + finish_list(Req, QServer, Etag, FoldResult, StartListRespFun, CurrentSeq, null) end). make_map_start_resp_fun(QueryServer, Db, LName) -> - fun(Req, Etag, TotalRows, Offset, _Acc) -> - Head = {[{<<"total_rows">>, TotalRows}, {<<"offset">>, Offset}]}, + fun(Req, Etag, TotalRows, Offset, _Acc, UpdateSeq) -> + Head = {[{<<"total_rows">>, TotalRows}, {<<"offset">>, Offset}, {<<"update_seq">>, UpdateSeq}]}, start_list_resp(QueryServer, LName, Req, Db, Head, Etag) end. make_reduce_start_resp_fun(QueryServer, Db, LName) -> - fun(Req2, Etag, _Acc) -> - start_list_resp(QueryServer, LName, Req2, Db, {[]}, Etag) + fun(Req2, Etag, _Acc, UpdateSeq) -> + start_list_resp(QueryServer, LName, Req2, Db, {[{<<"update_seq">>, UpdateSeq}]}, Etag) end. start_list_resp(QServer, LName, Req, Db, Head, Etag) -> JsonReq = couch_httpd_external:json_req_obj(Req, Db), - [<<"start">>,Chunks,JsonResp] = couch_query_servers:ddoc_proc_prompt(QServer, + [<<"start">>,Chunks,JsonResp] = couch_query_servers:ddoc_proc_prompt(QServer, [<<"lists">>, LName], [Head, JsonReq]), JsonResp2 = apply_etag(JsonResp, Etag), #extern_resp_args{ @@ -313,7 +348,7 @@ send_non_empty_chunk(Resp, Chunk) -> _ -> send_chunk(Resp, Chunk) end. -finish_list(Req, {Proc, _DDocId}, Etag, FoldResult, StartFun, TotalRows) -> +finish_list(Req, {Proc, _DDocId}, Etag, FoldResult, StartFun, CurrentSeq, TotalRows) -> FoldResult2 = case FoldResult of {Limit, SkipCount, Response, RowAcc} -> {Limit, SkipCount, Response, RowAcc, nil}; @@ -323,8 +358,8 @@ finish_list(Req, {Proc, _DDocId}, Etag, FoldResult, StartFun, TotalRows) -> case FoldResult2 of {_, _, undefined, _, _} -> {ok, Resp, BeginBody} = - render_head_for_empty_list(StartFun, Req, Etag, TotalRows), - [<<"end">>, Chunks] = couch_query_servers:proc_prompt(Proc, [<<"list_end">>]), + render_head_for_empty_list(StartFun, Req, Etag, CurrentSeq, TotalRows), + [<<"end">>, Chunks] = couch_query_servers:proc_prompt(Proc, [<<"list_end">>]), Chunk = BeginBody ++ ?b2l(?l2b(Chunks)), send_non_empty_chunk(Resp, Chunk); {_, _, Resp, stop, _} -> @@ -336,32 +371,17 @@ finish_list(Req, {Proc, _DDocId}, Etag, FoldResult, StartFun, TotalRows) -> last_chunk(Resp). -render_head_for_empty_list(StartListRespFun, Req, Etag, null) -> - StartListRespFun(Req, Etag, []); % for reduce -render_head_for_empty_list(StartListRespFun, Req, Etag, TotalRows) -> - StartListRespFun(Req, Etag, TotalRows, null, []). - - -% Maybe this is in the proplists API -% todo move to couch_util -json_apply_field(H, {L}) -> - json_apply_field(H, L, []). -json_apply_field({Key, NewValue}, [{Key, _OldVal} | Headers], Acc) -> - % drop matching keys - json_apply_field({Key, NewValue}, Headers, Acc); -json_apply_field({Key, NewValue}, [{OtherKey, OtherVal} | Headers], Acc) -> - % something else is next, leave it alone. - json_apply_field({Key, NewValue}, Headers, [{OtherKey, OtherVal} | Acc]); -json_apply_field({Key, NewValue}, [], Acc) -> - % end of list, add ours - {[{Key, NewValue}|Acc]}. +render_head_for_empty_list(StartListRespFun, Req, Etag, CurrentSeq, null) -> + StartListRespFun(Req, Etag, [], CurrentSeq); % for reduce +render_head_for_empty_list(StartListRespFun, Req, Etag, CurrentSeq, TotalRows) -> + StartListRespFun(Req, Etag, TotalRows, null, [], CurrentSeq). apply_etag({ExternalResponse}, CurrentEtag) -> % Here we embark on the delicate task of replacing or creating the % headers on the JsonResponse object. We need to control the Etag and % Vary headers. If the external function controls the Etag, we'd have to % run it to check for a match, which sort of defeats the purpose. - case proplists:get_value(<<"headers">>, ExternalResponse, nil) of + case couch_util:get_value(<<"headers">>, ExternalResponse, nil) of nil -> % no JSON headers % add our Etag and Vary headers to the response @@ -369,8 +389,8 @@ apply_etag({ExternalResponse}, CurrentEtag) -> JsonHeaders -> {[case Field of {<<"headers">>, JsonHeaders} -> % add our headers - JsonHeadersEtagged = json_apply_field({<<"Etag">>, CurrentEtag}, JsonHeaders), - JsonHeadersVaried = json_apply_field({<<"Vary">>, <<"Accept">>}, JsonHeadersEtagged), + JsonHeadersEtagged = couch_util:json_apply_field({<<"Etag">>, CurrentEtag}, JsonHeaders), + JsonHeadersVaried = couch_util:json_apply_field({<<"Vary">>, <<"Accept">>}, JsonHeadersEtagged), {<<"headers">>, JsonHeadersVaried}; _ -> % skip non-header fields Field diff --git a/src/couchdb/couch_httpd_stats_handlers.erl b/src/couchdb/couch_httpd_stats_handlers.erl index 40444bf8..41aeaed0 100644 --- a/src/couchdb/couch_httpd_stats_handlers.erl +++ b/src/couchdb/couch_httpd_stats_handlers.erl @@ -29,7 +29,8 @@ handle_stats_req(#httpd{method='GET', path_parts=[_, _Mod]}) -> handle_stats_req(#httpd{method='GET', path_parts=[_, Mod, Key]}=Req) -> flush(Req), - Stats = couch_stats_aggregator:get_json({?b2a(Mod), ?b2a(Key)}, range(Req)), + Stats = couch_stats_aggregator:get_json({list_to_atom(binary_to_list(Mod)), + list_to_atom(binary_to_list(Key))}, range(Req)), send_json(Req, {[{Mod, {[{Key, Stats}]}}]}); handle_stats_req(#httpd{method='GET', path_parts=[_, _Mod, _Key | _Extra]}) -> @@ -39,7 +40,7 @@ handle_stats_req(Req) -> send_method_not_allowed(Req, "GET"). range(Req) -> - case proplists:get_value("range", couch_httpd:qs(Req)) of + case couch_util:get_value("range", couch_httpd:qs(Req)) of undefined -> 0; Value -> @@ -47,7 +48,7 @@ range(Req) -> end. flush(Req) -> - case proplists:get_value("flush", couch_httpd:qs(Req)) of + case couch_util:get_value("flush", couch_httpd:qs(Req)) of "true" -> couch_stats_aggregator:collect_sample(); _Else -> diff --git a/src/couchdb/couch_httpd_view.erl b/src/couchdb/couch_httpd_view.erl index 6419ca55..cb387d1b 100644 --- a/src/couchdb/couch_httpd_view.erl +++ b/src/couchdb/couch_httpd_view.erl @@ -16,8 +16,8 @@ -export([handle_view_req/3,handle_temp_view_req/2]). -export([get_stale_type/1, get_reduce_type/1, parse_view_params/3]). --export([make_view_fold_fun/6, finish_view_fold/4, view_row_obj/3]). --export([view_group_etag/2, view_group_etag/3, make_reduce_fold_funs/5]). +-export([make_view_fold_fun/7, finish_view_fold/4, finish_view_fold/5, view_row_obj/3]). +-export([view_group_etag/2, view_group_etag/3, make_reduce_fold_funs/6]). -export([design_doc_view/5, parse_bool_param/1, doc_member/2]). -export([make_key_options/1, load_view/4]). @@ -26,6 +26,8 @@ start_json_response/2, start_json_response/3, end_json_response/1, send_chunked_error/2]). +-import(couch_db,[get_update_seq/1]). + design_doc_view(Req, Db, DName, ViewName, Keys) -> DesignId = <<"_design/", DName/binary>>, Stale = get_stale_type(Req), @@ -59,8 +61,9 @@ handle_view_req(#httpd{method='GET', handle_view_req(#httpd{method='POST', path_parts=[_, _, DName, _, ViewName]}=Req, Db, _DDoc) -> + couch_httpd:validate_ctype(Req, "application/json"), {Fields} = couch_httpd:json_body_obj(Req), - case proplists:get_value(<<"keys">>, Fields, nil) of + case couch_util:get_value(<<"keys">>, Fields, nil) of nil -> Fmt = "POST to view ~p/~p in database ~p with no keys member.", ?LOG_DEBUG(Fmt, [DName, ViewName, Db]), @@ -75,14 +78,16 @@ handle_view_req(Req, _Db, _DDoc) -> send_method_not_allowed(Req, "GET,POST,HEAD"). handle_temp_view_req(#httpd{method='POST'}=Req, Db) -> + couch_httpd:validate_ctype(Req, "application/json"), + ok = couch_db:check_is_admin(Db), couch_stats_collector:increment({httpd, temporary_view_reads}), {Props} = couch_httpd:json_body_obj(Req), - Language = proplists:get_value(<<"language">>, Props, <<"javascript">>), - {DesignOptions} = proplists:get_value(<<"options">>, Props, {[]}), - MapSrc = proplists:get_value(<<"map">>, Props), - Keys = proplists:get_value(<<"keys">>, Props, nil), + Language = couch_util:get_value(<<"language">>, Props, <<"javascript">>), + {DesignOptions} = couch_util:get_value(<<"options">>, Props, {[]}), + MapSrc = couch_util:get_value(<<"map">>, Props), + Keys = couch_util:get_value(<<"keys">>, Props, nil), Reduce = get_reduce_type(Req), - case proplists:get_value(<<"reduce">>, Props, null) of + case couch_util:get_value(<<"reduce">>, Props, null) of null -> QueryArgs = parse_view_params(Req, Keys, map), {ok, View, Group} = couch_view:get_temp_map_view(Db, Language, @@ -111,11 +116,11 @@ output_map_view(Req, View, Group, Db, QueryArgs, nil) -> CurrentEtag = view_group_etag(Group, Db), couch_httpd:etag_respond(Req, CurrentEtag, fun() -> {ok, RowCount} = couch_view:get_row_count(View), - FoldlFun = make_view_fold_fun(Req, QueryArgs, CurrentEtag, Db, RowCount, #view_fold_helper_funs{reduce_count=fun couch_view:reduce_to_count/1}), + FoldlFun = make_view_fold_fun(Req, QueryArgs, CurrentEtag, Db, Group#group.current_seq, RowCount, #view_fold_helper_funs{reduce_count=fun couch_view:reduce_to_count/1}), FoldAccInit = {Limit, SkipCount, undefined, []}, - {ok, LastReduce, FoldResult} = couch_view:fold(View, + {ok, LastReduce, FoldResult} = couch_view:fold(View, FoldlFun, FoldAccInit, make_key_options(QueryArgs)), - finish_view_fold(Req, RowCount, + finish_view_fold(Req, RowCount, couch_view:reduce_to_count(LastReduce), FoldResult) end); @@ -128,19 +133,19 @@ output_map_view(Req, View, Group, Db, QueryArgs, Keys) -> couch_httpd:etag_respond(Req, CurrentEtag, fun() -> {ok, RowCount} = couch_view:get_row_count(View), FoldAccInit = {Limit, SkipCount, undefined, []}, - {LastReduce, FoldResult} = lists:foldl( - fun(Key, {_, FoldAcc}) -> - FoldlFun = make_view_fold_fun(Req, - QueryArgs#view_query_args{ - }, CurrentEtag, Db, RowCount, + {LastReduce, FoldResult} = lists:foldl(fun(Key, {_, FoldAcc}) -> + FoldlFun = make_view_fold_fun(Req, QueryArgs#view_query_args{}, + CurrentEtag, Db, Group#group.current_seq, RowCount, #view_fold_helper_funs{ reduce_count = fun couch_view:reduce_to_count/1 }), - {ok, LastReduce, FoldResult} = couch_view:fold(View, FoldlFun, FoldAcc, - make_key_options(QueryArgs#view_query_args{start_key=Key, end_key=Key})), - {LastReduce, FoldResult} - end, {{[],[]}, FoldAccInit}, Keys), - finish_view_fold(Req, RowCount, couch_view:reduce_to_count(LastReduce), FoldResult) + {ok, LastReduce, FoldResult} = couch_view:fold(View, FoldlFun, + FoldAcc, make_key_options( + QueryArgs#view_query_args{start_key=Key, end_key=Key})), + {LastReduce, FoldResult} + end, {{[],[]}, FoldAccInit}, Keys), + finish_view_fold(Req, RowCount, couch_view:reduce_to_count(LastReduce), + FoldResult, [{update_seq,Group#group.current_seq}]) end). output_reduce_view(Req, Db, View, Group, QueryArgs, nil) -> @@ -151,7 +156,9 @@ output_reduce_view(Req, Db, View, Group, QueryArgs, nil) -> } = QueryArgs, CurrentEtag = view_group_etag(Group, Db), couch_httpd:etag_respond(Req, CurrentEtag, fun() -> - {ok, GroupRowsFun, RespFun} = make_reduce_fold_funs(Req, GroupLevel, QueryArgs, CurrentEtag, #reduce_fold_helper_funs{}), + {ok, GroupRowsFun, RespFun} = make_reduce_fold_funs(Req, GroupLevel, + QueryArgs, CurrentEtag, Group#group.current_seq, + #reduce_fold_helper_funs{}), FoldAccInit = {Limit, Skip, undefined, []}, {ok, {_, _, Resp, _}} = couch_view:fold_reduce(View, RespFun, FoldAccInit, [{key_group_fun, GroupRowsFun} | @@ -167,10 +174,13 @@ output_reduce_view(Req, Db, View, Group, QueryArgs, Keys) -> } = QueryArgs, CurrentEtag = view_group_etag(Group, Db, Keys), couch_httpd:etag_respond(Req, CurrentEtag, fun() -> - {ok, GroupRowsFun, RespFun} = make_reduce_fold_funs(Req, GroupLevel, QueryArgs, CurrentEtag, #reduce_fold_helper_funs{}), + {ok, GroupRowsFun, RespFun} = make_reduce_fold_funs(Req, GroupLevel, + QueryArgs, CurrentEtag, Group#group.current_seq, + #reduce_fold_helper_funs{}), {Resp, _RedAcc3} = lists:foldl( fun(Key, {Resp, RedAcc}) -> - % run the reduce once for each key in keys, with limit etc reapplied for each key + % run the reduce once for each key in keys, with limit etc + % reapplied for each key FoldAccInit = {Limit, Skip, Resp, RedAcc}, {_, {_, _, Resp2, RedAcc2}} = couch_view:fold_reduce(View, RespFun, FoldAccInit, [{key_group_fun, GroupRowsFun} | @@ -180,7 +190,7 @@ output_reduce_view(Req, Db, View, Group, QueryArgs, Keys) -> {Resp2, RedAcc2} end, {undefined, []}, Keys), % Start with no comma - finish_reduce_fold(Req, Resp) + finish_reduce_fold(Req, Resp, [{update_seq,Group#group.current_seq}]) end). reverse_key_default(?MIN_STR) -> ?MAX_STR; @@ -188,28 +198,28 @@ reverse_key_default(?MAX_STR) -> ?MIN_STR; reverse_key_default(Key) -> Key. get_stale_type(Req) -> - list_to_atom(couch_httpd:qs_value(Req, "stale", "nil")). + list_to_existing_atom(couch_httpd:qs_value(Req, "stale", "nil")). get_reduce_type(Req) -> - list_to_atom(couch_httpd:qs_value(Req, "reduce", "true")). + list_to_existing_atom(couch_httpd:qs_value(Req, "reduce", "true")). load_view(Req, Db, {ViewDesignId, ViewName}, Keys) -> - Stale = couch_httpd_view:get_stale_type(Req), - Reduce = couch_httpd_view:get_reduce_type(Req), + Stale = get_stale_type(Req), + Reduce = get_reduce_type(Req), case couch_view:get_map_view(Db, ViewDesignId, ViewName, Stale) of {ok, View, Group} -> - QueryArgs = couch_httpd_view:parse_view_params(Req, Keys, map), + QueryArgs = parse_view_params(Req, Keys, map), {map, View, Group, QueryArgs}; {not_found, _Reason} -> case couch_view:get_reduce_view(Db, ViewDesignId, ViewName, Stale) of {ok, ReduceView, Group} -> case Reduce of false -> - QueryArgs = couch_httpd_view:parse_view_params(Req, Keys, map_red), + QueryArgs = parse_view_params(Req, Keys, map_red), MapView = couch_view:extract_map_view(ReduceView), {map, MapView, Group, QueryArgs}; _ -> - QueryArgs = couch_httpd_view:parse_view_params(Req, Keys, reduce), + QueryArgs = parse_view_params(Req, Keys, reduce), {reduce, ReduceView, Group, QueryArgs} end; {not_found, Reason} -> @@ -226,8 +236,8 @@ parse_view_params(Req, Keys, ViewType) -> QueryList = couch_httpd:qs(Req), QueryParams = lists:foldl(fun({K, V}, Acc) -> - parse_view_param(K, V) ++ Acc - end, [], QueryList), + parse_view_param(K, V) ++ Acc + end, [], QueryList), IsMultiGet = (Keys =/= nil), Args = #view_query_args{ view_type=ViewType, @@ -239,17 +249,17 @@ parse_view_params(Req, Keys, ViewType) -> GroupLevel = QueryArgs#view_query_args.group_level, case {ViewType, GroupLevel, IsMultiGet} of - {reduce, exact, true} -> - QueryArgs; - {reduce, _, false} -> - QueryArgs; - {reduce, _, _} -> - % we can simplify code if we just drop this error message. - Msg = <<"Multi-key fetchs for reduce " - "view must include `group=true`">>, - throw({query_parse_error, Msg}); - _ -> - QueryArgs + {reduce, exact, true} -> + QueryArgs; + {reduce, _, false} -> + QueryArgs; + {reduce, _, _} -> + % we can simplify code if we just drop this error message. + Msg = <<"Multi-key fetchs for reduce " + "view must include `group=true`">>, + throw({query_parse_error, Msg}); + _ -> + QueryArgs end, QueryArgs. @@ -302,23 +312,23 @@ parse_view_param(Key, Value) -> validate_view_query(start_key, Value, Args) -> case Args#view_query_args.multi_get of - true -> - Msg = <<"Query parameter `start_key` is " - "not compatible with multi-get">>, - throw({query_parse_error, Msg}); - _ -> - Args#view_query_args{start_key=Value} + true -> + Msg = <<"Query parameter `start_key` is " + "not compatible with multi-get">>, + throw({query_parse_error, Msg}); + _ -> + Args#view_query_args{start_key=Value} end; validate_view_query(start_docid, Value, Args) -> Args#view_query_args{start_docid=Value}; validate_view_query(end_key, Value, Args) -> case Args#view_query_args.multi_get of - true-> - Msg = <<"Query parameter `end_key` is " - "not compatible with multi-get">>, - throw({query_parse_error, Msg}); - _ -> - Args#view_query_args{end_key=Value} + true-> + Msg = <<"Query parameter `end_key` is " + "not compatible with multi-get">>, + throw({query_parse_error, Msg}); + _ -> + Args#view_query_args{end_key=Value} end; validate_view_query(end_docid, Value, Args) -> Args#view_query_args{end_docid=Value}; @@ -330,15 +340,15 @@ validate_view_query(stale, _, Args) -> Args; validate_view_query(descending, true, Args) -> case Args#view_query_args.direction of - rev -> Args; % Already reversed - fwd -> - Args#view_query_args{ - direction = rev, - start_docid = - reverse_key_default(Args#view_query_args.start_docid), - end_docid = - reverse_key_default(Args#view_query_args.end_docid) - } + rev -> Args; % Already reversed + fwd -> + Args#view_query_args{ + direction = rev, + start_docid = + reverse_key_default(Args#view_query_args.start_docid), + end_docid = + reverse_key_default(Args#view_query_args.end_docid) + } end; validate_view_query(descending, false, Args) -> Args; % Ignore default condition @@ -346,38 +356,41 @@ validate_view_query(skip, Value, Args) -> Args#view_query_args{skip=Value}; validate_view_query(group_level, Value, Args) -> case Args#view_query_args.view_type of - reduce -> - Args#view_query_args{group_level=Value}; - _ -> - Msg = <<"Invalid URL parameter 'group' or " - " 'group_level' for non-reduce view.">>, - throw({query_parse_error, Msg}) + reduce -> + Args#view_query_args{group_level=Value}; + _ -> + Msg = <<"Invalid URL parameter 'group' or " + " 'group_level' for non-reduce view.">>, + throw({query_parse_error, Msg}) end; validate_view_query(inclusive_end, Value, Args) -> Args#view_query_args{inclusive_end=Value}; +validate_view_query(reduce, false, Args) -> + Args; validate_view_query(reduce, _, Args) -> case Args#view_query_args.view_type of - map -> - Msg = <<"Invalid URL parameter `reduce` for map view.">>, - throw({query_parse_error, Msg}); - _ -> - Args + map -> + Msg = <<"Invalid URL parameter `reduce` for map view.">>, + throw({query_parse_error, Msg}); + _ -> + Args end; validate_view_query(include_docs, true, Args) -> case Args#view_query_args.view_type of - reduce -> - Msg = <<"Query parameter `include_docs` " - "is invalid for reduce views.">>, - throw({query_parse_error, Msg}); - _ -> - Args#view_query_args{include_docs=true} + reduce -> + Msg = <<"Query parameter `include_docs` " + "is invalid for reduce views.">>, + throw({query_parse_error, Msg}); + _ -> + Args#view_query_args{include_docs=true} end; +% Use the view_query_args record's default value validate_view_query(include_docs, _Value, Args) -> Args; validate_view_query(extra, _Value, Args) -> Args. -make_view_fold_fun(Req, QueryArgs, Etag, Db, TotalViewCount, HelperFuns) -> +make_view_fold_fun(Req, QueryArgs, Etag, Db, UpdateSeq, TotalViewCount, HelperFuns) -> #view_fold_helper_funs{ start_response = StartRespFun, send_row = SendRowFun, @@ -388,7 +401,8 @@ make_view_fold_fun(Req, QueryArgs, Etag, Db, TotalViewCount, HelperFuns) -> include_docs = IncludeDocs } = QueryArgs, - fun({{Key, DocId}, Value}, OffsetReds, {AccLimit, AccSkip, Resp, RowFunAcc}) -> + fun({{Key, DocId}, Value}, OffsetReds, + {AccLimit, AccSkip, Resp, RowFunAcc}) -> case {AccLimit, AccSkip, Resp} of {0, _, _} -> % we've done "limit" rows, stop foldling @@ -400,7 +414,7 @@ make_view_fold_fun(Req, QueryArgs, Etag, Db, TotalViewCount, HelperFuns) -> % rendering the first row, first we start the response Offset = ReduceCountFun(OffsetReds), {ok, Resp2, RowFunAcc0} = StartRespFun(Req, Etag, - TotalViewCount, Offset, RowFunAcc), + TotalViewCount, Offset, RowFunAcc, UpdateSeq), {Go, RowFunAcc2} = SendRowFun(Resp2, Db, {{Key, DocId}, Value}, IncludeDocs, RowFunAcc0), {Go, {AccLimit - 1, 0, Resp2, RowFunAcc2}}; @@ -412,7 +426,7 @@ make_view_fold_fun(Req, QueryArgs, Etag, Db, TotalViewCount, HelperFuns) -> end end. -make_reduce_fold_funs(Req, GroupLevel, _QueryArgs, Etag, HelperFuns) -> +make_reduce_fold_funs(Req, GroupLevel, _QueryArgs, Etag, UpdateSeq, HelperFuns) -> #reduce_fold_helper_funs{ start_response = StartRespFun, send_row = SendRowFun @@ -438,7 +452,7 @@ make_reduce_fold_funs(Req, GroupLevel, _QueryArgs, Etag, HelperFuns) -> (_Key, Red, {AccLimit, 0, undefined, RowAcc0}) when GroupLevel == 0 -> % we haven't started responding yet and group=false - {ok, Resp2, RowAcc} = StartRespFun(Req, Etag, RowAcc0), + {ok, Resp2, RowAcc} = StartRespFun(Req, Etag, RowAcc0, UpdateSeq), {Go, RowAcc2} = SendRowFun(Resp2, {null, Red}, RowAcc), {Go, {AccLimit - 1, 0, Resp2, RowAcc2}}; (_Key, Red, {AccLimit, 0, Resp, RowAcc}) when GroupLevel == 0 -> @@ -449,18 +463,20 @@ make_reduce_fold_funs(Req, GroupLevel, _QueryArgs, Etag, HelperFuns) -> (Key, Red, {AccLimit, 0, undefined, RowAcc0}) when is_integer(GroupLevel), is_list(Key) -> % group_level and we haven't responded yet - {ok, Resp2, RowAcc} = StartRespFun(Req, Etag, RowAcc0), - {Go, RowAcc2} = SendRowFun(Resp2, {lists:sublist(Key, GroupLevel), Red}, RowAcc), + {ok, Resp2, RowAcc} = StartRespFun(Req, Etag, RowAcc0, UpdateSeq), + {Go, RowAcc2} = SendRowFun(Resp2, + {lists:sublist(Key, GroupLevel), Red}, RowAcc), {Go, {AccLimit - 1, 0, Resp2, RowAcc2}}; (Key, Red, {AccLimit, 0, Resp, RowAcc}) when is_integer(GroupLevel), is_list(Key) -> % group_level and we've already started the response - {Go, RowAcc2} = SendRowFun(Resp, {lists:sublist(Key, GroupLevel), Red}, RowAcc), + {Go, RowAcc2} = SendRowFun(Resp, + {lists:sublist(Key, GroupLevel), Red}, RowAcc), {Go, {AccLimit - 1, 0, Resp, RowAcc2}}; (Key, Red, {AccLimit, 0, undefined, RowAcc0}) -> % group=true and we haven't responded yet - {ok, Resp2, RowAcc} = StartRespFun(Req, Etag, RowAcc0), + {ok, Resp2, RowAcc} = StartRespFun(Req, Etag, RowAcc0, UpdateSeq), {Go, RowAcc2} = SendRowFun(Resp2, {Key, Red}, RowAcc), {Go, {AccLimit - 1, 0, Resp2, RowAcc2}}; (Key, Red, {AccLimit, 0, Resp, RowAcc}) -> @@ -470,13 +486,13 @@ make_reduce_fold_funs(Req, GroupLevel, _QueryArgs, Etag, HelperFuns) -> end, {ok, GroupRowsFun, RespFun}. -apply_default_helper_funs(#view_fold_helper_funs{ - start_response = StartResp, - send_row = SendRow -}=Helpers) -> - +apply_default_helper_funs( + #view_fold_helper_funs{ + start_response = StartResp, + send_row = SendRow + }=Helpers) -> StartResp2 = case StartResp of - undefined -> fun json_view_start_resp/5; + undefined -> fun json_view_start_resp/6; _ -> StartResp end, @@ -491,12 +507,13 @@ apply_default_helper_funs(#view_fold_helper_funs{ }; -apply_default_helper_funs(#reduce_fold_helper_funs{ - start_response = StartResp, - send_row = SendRow -}=Helpers) -> +apply_default_helper_funs( + #reduce_fold_helper_funs{ + start_response = StartResp, + send_row = SendRow + }=Helpers) -> StartResp2 = case StartResp of - undefined -> fun json_reduce_start_resp/3; + undefined -> fun json_reduce_start_resp/4; _ -> StartResp end, @@ -511,7 +528,7 @@ apply_default_helper_funs(#reduce_fold_helper_funs{ }. make_key_options(#view_query_args{direction = Dir}=QueryArgs) -> - [{dir,Dir} | make_start_key_option(QueryArgs) ++ + [{dir,Dir} | make_start_key_option(QueryArgs) ++ make_end_key_option(QueryArgs)]. make_start_key_option( @@ -538,10 +555,19 @@ make_end_key_option( inclusive_end = false}) -> [{end_key_gt, {EndKey,reverse_key_default(EndDocId)}}]. -json_view_start_resp(Req, Etag, TotalViewCount, Offset, _Acc) -> +json_view_start_resp(Req, Etag, TotalViewCount, Offset, _Acc, UpdateSeq) -> {ok, Resp} = start_json_response(Req, 200, [{"Etag", Etag}]), - BeginBody = io_lib:format("{\"total_rows\":~w,\"offset\":~w,\"rows\":[\r\n", - [TotalViewCount, Offset]), + BeginBody = case couch_httpd:qs_value(Req, "update_seq") of + "true" -> + io_lib:format( + "{\"total_rows\":~w,\"update_seq\":~w," + "\"offset\":~w,\"rows\":[\r\n", + [TotalViewCount, UpdateSeq, Offset]); + _Else -> + io_lib:format( + "{\"total_rows\":~w,\"offset\":~w,\"rows\":[\r\n", + [TotalViewCount, Offset]) + end, {ok, Resp, BeginBody}. send_json_view_row(Resp, Db, {{Key, DocId}, Value}, IncludeDocs, RowFront) -> @@ -549,9 +575,14 @@ send_json_view_row(Resp, Db, {{Key, DocId}, Value}, IncludeDocs, RowFront) -> send_chunk(Resp, RowFront ++ ?JSON_ENCODE(JsonObj)), {ok, ",\r\n"}. -json_reduce_start_resp(Req, Etag, _Acc0) -> +json_reduce_start_resp(Req, Etag, _Acc0, UpdateSeq) -> {ok, Resp} = start_json_response(Req, 200, [{"Etag", Etag}]), - {ok, Resp, "{\"rows\":[\r\n"}. + case couch_httpd:qs_value(Req, "update_seq") of + "true" -> + {ok, Resp, io_lib:format("{\"update_seq\":~w,\"rows\":[\r\n",[UpdateSeq])}; + _Else -> + {ok, Resp, "{\"rows\":[\r\n"} + end. send_json_reduce_row(Resp, {Key, Value}, RowFront) -> send_chunk(Resp, RowFront ++ ?JSON_ENCODE({[{key, Key}, {value, Value}]})), @@ -573,13 +604,13 @@ view_row_obj(_Db, {{Key, error}, Value}, _IncludeDocs) -> {[{key, Key}, {error, Value}]}; % include docs in the view output view_row_obj(Db, {{Key, DocId}, {Props}}, true) -> - Rev = case proplists:get_value(<<"_rev">>, Props) of + Rev = case couch_util:get_value(<<"_rev">>, Props) of undefined -> nil; Rev0 -> couch_doc:parse_rev(Rev0) end, - IncludeId = proplists:get_value(<<"_id">>, Props, DocId), + IncludeId = couch_util:get_value(<<"_id">>, Props, DocId), view_row_with_doc(Db, {{Key, DocId}, {Props}}, {IncludeId, Rev}); view_row_obj(Db, {{Key, DocId}, Value}, true) -> view_row_with_doc(Db, {{Key, DocId}, Value}, {DocId, nil}); @@ -593,15 +624,17 @@ view_row_with_doc(Db, {{Key, DocId}, Value}, IdRev) -> doc_member(Db, {DocId, Rev}) -> ?LOG_DEBUG("Include Doc: ~p ~p", [DocId, Rev]), case (catch couch_httpd_db:couch_doc_open(Db, DocId, Rev, [])) of - #doc{} = Doc -> - JsonDoc = couch_doc:to_json_obj(Doc, []), - [{doc, JsonDoc}]; - _Else -> - [{doc, null}] + #doc{} = Doc -> + JsonDoc = couch_doc:to_json_obj(Doc, []), + [{doc, JsonDoc}]; + _Else -> + [{doc, null}] end. - finish_view_fold(Req, TotalRows, Offset, FoldResult) -> + finish_view_fold(Req, TotalRows, Offset, FoldResult, []). + +finish_view_fold(Req, TotalRows, Offset, FoldResult, Fields) -> case FoldResult of {_, _, undefined, _} -> % nothing found in the view or keys, nothing has been returned @@ -610,7 +643,7 @@ finish_view_fold(Req, TotalRows, Offset, FoldResult) -> {total_rows, TotalRows}, {offset, Offset}, {rows, []} - ]}); + ] ++ Fields}); {_, _, Resp, _} -> % end the view send_chunk(Resp, "\r\n]}"), @@ -618,11 +651,14 @@ finish_view_fold(Req, TotalRows, Offset, FoldResult) -> end. finish_reduce_fold(Req, Resp) -> + finish_reduce_fold(Req, Resp, []). + +finish_reduce_fold(Req, Resp, Fields) -> case Resp of undefined -> send_json(Req, 200, {[ {rows, []} - ]}); + ] ++ Fields}); Resp -> send_chunk(Resp, "\r\n]}"), end_json_response(Resp) @@ -630,11 +666,11 @@ finish_reduce_fold(Req, Resp) -> parse_bool_param(Val) -> case string:to_lower(Val) of - "true" -> true; - "false" -> false; - _ -> - Msg = io_lib:format("Invalid boolean parameter: ~p", [Val]), - throw({query_parse_error, ?l2b(Msg)}) + "true" -> true; + "false" -> false; + _ -> + Msg = io_lib:format("Invalid boolean parameter: ~p", [Val]), + throw({query_parse_error, ?l2b(Msg)}) end. parse_int_param(Val) -> diff --git a/src/couchdb/couch_js_functions.hrl b/src/couchdb/couch_js_functions.hrl new file mode 100644 index 00000000..32573a90 --- /dev/null +++ b/src/couchdb/couch_js_functions.hrl @@ -0,0 +1,97 @@ +% 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. + +-define(AUTH_DB_DOC_VALIDATE_FUNCTION, <<" + function(newDoc, oldDoc, userCtx) { + if (newDoc._deleted === true) { + // allow deletes by admins and matching users + // without checking the other fields + if ((userCtx.roles.indexOf('_admin') !== -1) || + (userCtx.name == oldDoc.name)) { + return; + } else { + throw({forbidden: 'Only admins may delete other user docs.'}); + } + } + + if ((oldDoc && oldDoc.type !== 'user') || newDoc.type !== 'user') { + throw({forbidden : 'doc.type must be user'}); + } // we only allow user docs for now + + if (!newDoc.name) { + throw({forbidden: 'doc.name is required'}); + } + + if (newDoc.roles && !isArray(newDoc.roles)) { + throw({forbidden: 'doc.roles must be an array'}); + } + + if (newDoc._id !== ('org.couchdb.user:' + newDoc.name)) { + throw({ + forbidden: 'Doc ID must be of the form org.couchdb.user:name' + }); + } + + if (oldDoc) { // validate all updates + if (oldDoc.name !== newDoc.name) { + throw({forbidden: 'Usernames can not be changed.'}); + } + } + + if (newDoc.password_sha && !newDoc.salt) { + throw({ + forbidden: 'Users with password_sha must have a salt.' + + 'See /_utils/script/couch.js for example code.' + }); + } + + if (userCtx.roles.indexOf('_admin') === -1) { + if (oldDoc) { // validate non-admin updates + if (userCtx.name !== newDoc.name) { + throw({ + forbidden: 'You may only update your own user document.' + }); + } + // validate role updates + var oldRoles = oldDoc.roles.sort(); + var newRoles = newDoc.roles.sort(); + + if (oldRoles.length !== newRoles.length) { + throw({forbidden: 'Only _admin may edit roles'}); + } + + for (var i = 0; i < oldRoles.length; i++) { + if (oldRoles[i] !== newRoles[i]) { + throw({forbidden: 'Only _admin may edit roles'}); + } + } + } else if (newDoc.roles.length > 0) { + throw({forbidden: 'Only _admin may set roles'}); + } + } + + // no system roles in users db + for (var i = 0; i < newDoc.roles.length; i++) { + if (newDoc.roles[i][0] === '_') { + throw({ + forbidden: + 'No system roles (starting with underscore) in users db.' + }); + } + } + + // no system names as names + if (newDoc.name[0] === '_') { + throw({forbidden: 'Username may not start with underscore.'}); + } + } +">>). diff --git a/src/couchdb/couch_key_tree.erl b/src/couchdb/couch_key_tree.erl index d5944119..985aebc0 100644 --- a/src/couchdb/couch_key_tree.erl +++ b/src/couchdb/couch_key_tree.erl @@ -16,100 +16,97 @@ -export([map/2, get_all_leafs/1, count_leafs/1, remove_leafs/2, get_all_leafs_full/1,stem/2,map_leafs/2]). -% a key tree looks like this: -% Tree -> [] or [{Key, Value, ChildTree} | SiblingTree] -% ChildTree -> Tree -% SiblingTree -> [] or [{SiblingKey, Value, Tree} | Tree] -% And each Key < SiblingKey - +% Tree::term() is really a tree(), but we don't want to require R13B04 yet +-type branch() :: {Key::term(), Value::term(), Tree::term()}. +-type path() :: {Start::pos_integer(), branch()}. +-type tree() :: [branch()]. % sorted by key % partial trees arranged by how much they are cut off. -merge(A, B) -> - {Merged, HasConflicts} = - lists:foldl( - fun(InsertTree, {AccTrees, AccConflicts}) -> - {ok, Merged, Conflicts} = merge_one(AccTrees, InsertTree, [], false), - {Merged, Conflicts or AccConflicts} - end, - {A, false}, B), - if HasConflicts or - ((length(Merged) =/= length(A)) and (length(Merged) =/= length(B))) -> +-spec merge([path()], path()) -> {[path()], conflicts | no_conflicts}. +merge(Paths, Path) -> + {ok, Merged, HasConflicts} = merge_one(Paths, Path, [], false), + if HasConflicts -> + Conflicts = conflicts; + (length(Merged) =/= length(Paths)) and (length(Merged) =/= 1) -> Conflicts = conflicts; true -> Conflicts = no_conflicts end, {lists:sort(Merged), Conflicts}. +-spec merge_one(Original::[path()], Inserted::path(), [path()], bool()) -> + {ok, Merged::[path()], NewConflicts::bool()}. merge_one([], Insert, OutAcc, ConflictsAcc) -> {ok, [Insert | OutAcc], ConflictsAcc}; -merge_one([{Start, Tree}|Rest], {StartInsert, TreeInsert}, OutAcc, ConflictsAcc) -> - if Start =< StartInsert -> - StartA = Start, - StartB = StartInsert, - TreeA = Tree, - TreeB = TreeInsert; - true -> - StartB = Start, - StartA = StartInsert, - TreeB = Tree, - TreeA = TreeInsert - end, - case merge_at([TreeA], StartB - StartA, TreeB) of - {ok, [CombinedTrees], Conflicts} -> - merge_one(Rest, {StartA, CombinedTrees}, OutAcc, Conflicts or ConflictsAcc); +merge_one([{Start, Tree}|Rest], {StartInsert, TreeInsert}, Acc, HasConflicts) -> + case merge_at([Tree], StartInsert - Start, [TreeInsert]) of + {ok, [Merged], Conflicts} -> + MergedStart = lists:min([Start, StartInsert]), + merge_one(Rest, {MergedStart, Merged}, Acc, Conflicts or HasConflicts); no -> - merge_one(Rest, {StartB, TreeB}, [{StartA, TreeA} | OutAcc], ConflictsAcc) + AccOut = [{Start, Tree} | Acc], + merge_one(Rest, {StartInsert, TreeInsert}, AccOut, HasConflicts) end. +-spec merge_at(tree(), Place::integer(), tree()) -> + {ok, Merged::tree(), HasConflicts::bool()} | no. +merge_at(_Ours, _Place, []) -> + no; merge_at([], _Place, _Insert) -> no; -merge_at([{Key, Value, SubTree}|Sibs], 0, {InsertKey, InsertValue, InsertSubTree}) -> - if Key == InsertKey -> - {Merge, Conflicts} = merge_simple(SubTree, InsertSubTree), - {ok, [{Key, Value, Merge} | Sibs], Conflicts}; - true -> - case merge_at(Sibs, 0, {InsertKey, InsertValue, InsertSubTree}) of - {ok, Merged, Conflicts} -> - {ok, [{Key, Value, SubTree} | Merged], Conflicts}; - no -> - no - end - end; -merge_at([{Key, Value, SubTree}|Sibs], Place, Insert) -> - case merge_at(SubTree, Place - 1,Insert) of +merge_at([{Key, Value, SubTree}|Sibs], Place, InsertTree) when Place > 0 -> + % inserted starts later than committed, need to drill into committed subtree + case merge_at(SubTree, Place - 1, InsertTree) of {ok, Merged, Conflicts} -> {ok, [{Key, Value, Merged} | Sibs], Conflicts}; no -> - case merge_at(Sibs, Place, Insert) of + case merge_at(Sibs, Place, InsertTree) of {ok, Merged, Conflicts} -> {ok, [{Key, Value, SubTree} | Merged], Conflicts}; no -> no end + end; +merge_at(OurTree, Place, [{Key, Value, SubTree}]) when Place < 0 -> + % inserted starts earlier than committed, need to drill into insert subtree + case merge_at(OurTree, Place + 1, SubTree) of + {ok, Merged, Conflicts} -> + {ok, [{Key, Value, Merged}], Conflicts}; + no -> + no + end; +merge_at([{Key, Value, SubTree}|Sibs], 0, [{Key, _Value, InsertSubTree}]) -> + {Merged, Conflicts} = merge_simple(SubTree, InsertSubTree), + {ok, [{Key, Value, Merged} | Sibs], Conflicts}; +merge_at([{OurKey, _, _} | _], 0, [{Key, _, _}]) when OurKey > Key -> + % siblings keys are ordered, no point in continuing + no; +merge_at([Tree | Sibs], 0, InsertTree) -> + case merge_at(Sibs, 0, InsertTree) of + {ok, Merged, Conflicts} -> + {ok, [Tree | Merged], Conflicts}; + no -> + no end. % key tree functions + +-spec merge_simple(tree(), tree()) -> {Merged::tree(), NewConflicts::bool()}. merge_simple([], B) -> {B, false}; merge_simple(A, []) -> {A, false}; -merge_simple([ATree | ANextTree], [BTree | BNextTree]) -> - {AKey, AValue, ASubTree} = ATree, - {BKey, _BValue, BSubTree} = BTree, - if - AKey == BKey -> - %same key - {MergedSubTree, Conflict1} = merge_simple(ASubTree, BSubTree), - {MergedNextTree, Conflict2} = merge_simple(ANextTree, BNextTree), - {[{AKey, AValue, MergedSubTree} | MergedNextTree], Conflict1 or Conflict2}; - AKey < BKey -> - {MTree, _} = merge_simple(ANextTree, [BTree | BNextTree]), - {[ATree | MTree], true}; - true -> - {MTree, _} = merge_simple([ATree | ANextTree], BNextTree), - {[BTree | MTree], true} - end. +merge_simple([{Key, Value, SubA} | NextA], [{Key, _, SubB} | NextB]) -> + {MergedSubTree, Conflict1} = merge_simple(SubA, SubB), + {MergedNextTree, Conflict2} = merge_simple(NextA, NextB), + {[{Key, Value, MergedSubTree} | MergedNextTree], Conflict1 or Conflict2}; +merge_simple([{A, _, _} = Tree | Next], [{B, _, _} | _] = Insert) when A < B -> + {Merged, _} = merge_simple(Next, Insert), + {[Tree | Merged], true}; +merge_simple(Ours, [Tree | Next]) -> + {Merged, _} = merge_simple(Ours, Next), + {[Tree | Merged], true}. find_missing(_Tree, []) -> []; @@ -159,7 +156,7 @@ remove_leafs(Trees, Keys) -> fun({PathPos, Path},TreeAcc) -> [SingleTree] = lists:foldl( fun({K,V},NewTreeAcc) -> [{K,V,NewTreeAcc}] end, [], Path), - {NewTrees, _} = merge(TreeAcc, [{PathPos + 1 - length(Path), SingleTree}]), + {NewTrees, _} = merge(TreeAcc, {PathPos + 1 - length(Path), SingleTree}), NewTrees end, [], FilteredPaths), {NewTree, RemovedKeys}. @@ -290,7 +287,7 @@ map(Fun, [{Pos, Tree}|Rest]) -> map_simple(_Fun, _Pos, []) -> []; map_simple(Fun, Pos, [{Key, Value, SubTree} | RestTree]) -> - Value2 = Fun({Pos, Key}, Value, + Value2 = Fun({Pos, Key}, Value, if SubTree == [] -> leaf; true -> branch end), [{Key, Value2, map_simple(Fun, Pos + 1, SubTree)} | map_simple(Fun, Pos, RestTree)]. @@ -321,7 +318,7 @@ stem(Trees, Limit) -> fun({PathPos, Path},TreeAcc) -> [SingleTree] = lists:foldl( fun({K,V},NewTreeAcc) -> [{K,V,NewTreeAcc}] end, [], Path), - {NewTrees, _} = merge(TreeAcc, [{PathPos + 1 - length(Path), SingleTree}]), + {NewTrees, _} = merge(TreeAcc, {PathPos + 1 - length(Path), SingleTree}), NewTrees end, [], Paths2). diff --git a/src/couchdb/couch_log.erl b/src/couchdb/couch_log.erl index 5a45c207..2d62cbb5 100644 --- a/src/couchdb/couch_log.erl +++ b/src/couchdb/couch_log.erl @@ -50,14 +50,23 @@ init([]) -> fun("log", "file") -> ?MODULE:stop(); ("log", "level") -> + ?MODULE:stop(); + ("log", "include_sasl") -> ?MODULE:stop() end), Filename = couch_config:get("log", "file", "couchdb.log"), - Level = couch_config:get("log", "level", "info"), + Level = level_integer(list_to_atom(couch_config:get("log", "level", "info"))), + Sasl = list_to_atom(couch_config:get("log", "include_sasl", "true")), + + case ets:info(?MODULE) of + undefined -> ets:new(?MODULE, [named_table]); + _ -> ok + end, + ets:insert(?MODULE, {level, Level}), {ok, Fd} = file:open(Filename, [append]), - {ok, {Fd, level_integer(list_to_atom(Level))}}. + {ok, {Fd, Level, Sasl}}. debug_on() -> get_level_integer() =< ?LEVEL_DEBUG. @@ -72,29 +81,35 @@ get_level() -> level_atom(get_level_integer()). get_level_integer() -> - catch gen_event:call(error_logger, couch_log, get_level_integer). + try + ets:lookup_element(?MODULE, level, 2) + catch error:badarg -> + ?LEVEL_ERROR + end. set_level_integer(Int) -> gen_event:call(error_logger, couch_log, {set_level_integer, Int}). -handle_event({error_report, _, {Pid, couch_error, {Format, Args}}}, {Fd, _LogLevel}=State) -> - log(Fd, Pid, error, Format, Args), - {ok, State}; -handle_event({error_report, _, {Pid, _, _}}=Event, {Fd, _LogLevel}=State) -> - log(Fd, Pid, error, "~p", [Event]), - {ok, State}; -handle_event({error, _, {Pid, Format, Args}}, {Fd, _LogLevel}=State) -> +handle_event({Pid, couch_error, {Format, Args}}, {Fd, _LogLevel, _Sasl}=State) -> log(Fd, Pid, error, Format, Args), {ok, State}; -handle_event({info_report, _, {Pid, couch_info, {Format, Args}}}, {Fd, LogLevel}=State) +handle_event({Pid, couch_info, {Format, Args}}, {Fd, LogLevel, _Sasl}=State) when LogLevel =< ?LEVEL_INFO -> log(Fd, Pid, info, Format, Args), {ok, State}; -handle_event({info_report, _, {Pid, couch_debug, {Format, Args}}}, {Fd, LogLevel}=State) +handle_event({Pid, couch_debug, {Format, Args}}, {Fd, LogLevel, _Sasl}=State) when LogLevel =< ?LEVEL_DEBUG -> log(Fd, Pid, debug, Format, Args), {ok, State}; -handle_event({_, _, {Pid, _, _}}=Event, {Fd, LogLevel}=State) +handle_event({error_report, _, {Pid, _, _}}=Event, {Fd, _LogLevel, Sasl}=State) +when Sasl =/= false -> + log(Fd, Pid, error, "~p", [Event]), + {ok, State}; +handle_event({error, _, {Pid, Format, Args}}, {Fd, _LogLevel, Sasl}=State) +when Sasl =/= false -> + log(Fd, Pid, error, Format, Args), + {ok, State}; +handle_event({_, _, {Pid, _, _}}=Event, {Fd, LogLevel, _Sasl}=State) when LogLevel =< ?LEVEL_TMI -> % log every remaining event if tmi! log(Fd, Pid, tmi, "~p", [Event]), @@ -102,10 +117,9 @@ when LogLevel =< ?LEVEL_TMI -> handle_event(_Event, State) -> {ok, State}. -handle_call(get_level_integer, {_Fd, LogLevel}=State) -> - {ok, LogLevel, State}; -handle_call({set_level_integer, NewLevel}, {Fd, _LogLevel}) -> - {ok, ok, {Fd, NewLevel}}. +handle_call({set_level_integer, NewLevel}, {Fd, _LogLevel, Sasl}) -> + ets:insert(?MODULE, {level, NewLevel}), + {ok, ok, {Fd, NewLevel, Sasl}}. handle_info(_Info, State) -> {ok, State}. @@ -113,7 +127,7 @@ handle_info(_Info, State) -> code_change(_OldVsn, State, _Extra) -> {ok, State}. -terminate(_Arg, {Fd, _LoggingLevel}) -> +terminate(_Arg, {Fd, _LoggingLevel, _Sasl}) -> file:close(Fd). log(Fd, Pid, Level, Format, Args) -> diff --git a/src/couchdb/couch_native_process.erl b/src/couchdb/couch_native_process.erl index 65e4e131..b512f712 100644 --- a/src/couchdb/couch_native_process.erl +++ b/src/couchdb/couch_native_process.erl @@ -1,16 +1,16 @@ -% Licensed under the Apache License, Version 2.0 (the "License"); -% you may not use this file except in compliance with the License. +% 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. +% 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. +% and limitations under the License. % % This file drew much inspiration from erlview, which was written by and % copyright Michael McDaniel [http://autosys.us], and is also under APL 2.0 @@ -25,14 +25,14 @@ % % fun({Doc}) -> % % Below, we emit a single record - the _id as key, null as value -% DocId = proplists:get_value(Doc, <<"_id">>, null), +% DocId = couch_util:get_value(Doc, <<"_id">>, null), % Emit(DocId, null) % end. % % which should be roughly the same as the javascript: % emit(doc._id, null); % -% This module exposes enough functions such that a native erlang server can +% This module exposes enough functions such that a native erlang server can % act as a fully-fleged view server, but no 'helper' functions specifically % for simplifying your erlang view code. It is expected other third-party % extensions will evolve which offer useful layers on top of this view server @@ -40,7 +40,8 @@ -module(couch_native_process). -behaviour(gen_server). --export([start_link/0,init/1,terminate/2,handle_call/3,handle_cast/2]). +-export([start_link/0,init/1,terminate/2,handle_call/3,handle_cast/2,code_change/3, + handle_info/2]). -export([set_timeout/2, prompt/2]). -define(STATE, native_proc_state). @@ -76,23 +77,27 @@ handle_call({prompt, Data}, _From, State) -> throw:{error, Why} -> {State, [<<"error">>, Why, Why]} end, - + case Resp of {error, Reason} -> Msg = io_lib:format("couch native server error: ~p", [Reason]), {reply, [<<"error">>, <<"native_query_server">>, list_to_binary(Msg)], NewState}; [<<"error">> | Rest] -> - Msg = io_lib:format("couch native server error: ~p", [Rest]), + % Msg = io_lib:format("couch native server error: ~p", [Rest]), + % TODO: markh? (jan) {reply, [<<"error">> | Rest], NewState}; [<<"fatal">> | Rest] -> - Msg = io_lib:format("couch native server error: ~p", [Rest]), + % Msg = io_lib:format("couch native server error: ~p", [Rest]), + % TODO: markh? (jan) {stop, fatal, [<<"error">> | Rest], NewState}; Resp -> {reply, Resp, NewState} end. -handle_cast(_Msg, State) -> {noreply, State}. -handle_info(_Msg, State) -> {noreply, State}. +handle_cast(foo, State) -> {noreply, State}. +handle_info({'EXIT',_,normal}, State) -> {noreply, State}; +handle_info({'EXIT',_,Reason}, State) -> + {stop, Reason, State}. terminate(_Reason, _State) -> ok. code_change(_OldVersion, State, _Extra) -> {ok, State}. @@ -168,12 +173,12 @@ ddoc(State, {DDoc}, [FunPath, Args]) -> % load fun from the FunPath BFun = lists:foldl(fun (Key, {Props}) when is_list(Props) -> - proplists:get_value(Key, Props, nil); - (Key, Fun) when is_binary(Fun) -> + couch_util:get_value(Key, Props, nil); + (_Key, Fun) when is_binary(Fun) -> Fun; - (Key, nil) -> + (_Key, nil) -> throw({error, not_found}); - (Key, Fun) -> + (_Key, _Fun) -> throw({error, malformed_ddoc}) end, {DDoc}, FunPath), ddoc(State, makefun(State, BFun, {DDoc}), FunPath, Args). @@ -238,7 +243,7 @@ load_ddoc(DDocs, DDocId) -> try dict:fetch(DDocId, DDocs) of {DDoc} -> {DDoc} catch - _:Else -> throw({error, ?l2b(io_lib:format("Native Query Server missing DDoc with Id: ~s",[DDocId]))}) + _:_Else -> throw({error, ?l2b(io_lib:format("Native Query Server missing DDoc with Id: ~s",[DDocId]))}) end. bindings(State, Sig) -> @@ -300,7 +305,7 @@ bindings(State, Sig, DDoc) -> {'FoldRows', FoldRows} ], case DDoc of - {Props} -> + {_Props} -> Bindings ++ [{'DDoc', DDoc}]; _Else -> Bindings end. @@ -308,11 +313,11 @@ bindings(State, Sig, DDoc) -> % thanks to erlview, via: % http://erlang.org/pipermail/erlang-questions/2003-November/010544.html makefun(State, Source) -> - Sig = erlang:md5(Source), + Sig = couch_util:md5(Source), BindFuns = bindings(State, Sig), {Sig, makefun(State, Source, BindFuns)}. makefun(State, Source, {DDoc}) -> - Sig = erlang:md5(lists:flatten([Source, term_to_binary(DDoc)])), + Sig = couch_util:md5(lists:flatten([Source, term_to_binary(DDoc)])), BindFuns = bindings(State, Sig, {DDoc}), {Sig, makefun(State, Source, BindFuns)}; makefun(_State, Source, BindFuns) when is_list(BindFuns) -> @@ -365,7 +370,7 @@ start_list_resp(Self, Sig) -> undefined -> {[{<<"headers">>, {[]}}]}; CurrHdrs -> CurrHdrs end, - Chunks = + Chunks = case erlang:get(Sig) of undefined -> []; CurrChunks -> CurrChunks diff --git a/src/couchdb/couch_os_process.erl b/src/couchdb/couch_os_process.erl index 5ac13715..5776776b 100644 --- a/src/couchdb/couch_os_process.erl +++ b/src/couchdb/couch_os_process.erl @@ -93,10 +93,10 @@ readjson(OsProc) when is_record(OsProc, os_proc) -> ?LOG_INFO("OS Process ~p Log :: ~s", [OsProc#os_proc.port, Msg]), readjson(OsProc); [<<"error">>, Id, Reason] -> - throw({list_to_atom(binary_to_list(Id)),Reason}); + throw({couch_util:to_existing_atom(Id),Reason}); [<<"fatal">>, Id, Reason] -> ?LOG_INFO("OS Process ~p Fatal Error :: ~s ~p",[OsProc#os_proc.port, Id, Reason]), - throw({list_to_atom(binary_to_list(Id)),Reason}); + throw({couch_util:to_existing_atom(Id),Reason}); Result -> Result end. @@ -104,6 +104,7 @@ readjson(OsProc) when is_record(OsProc, os_proc) -> % gen_server API init([Command, Options, PortOptions]) -> + process_flag(trap_exit, true), PrivDir = couch_util:priv_dir(), Spawnkiller = filename:join(PrivDir, "couchspawnkillable"), BaseProc = #os_proc{ diff --git a/src/couchdb/couch_query_servers.erl b/src/couchdb/couch_query_servers.erl index 30f4c4c7..5f97cbd3 100644 --- a/src/couchdb/couch_query_servers.erl +++ b/src/couchdb/couch_query_servers.erl @@ -15,9 +15,9 @@ -export([start_link/0]). --export([init/1, terminate/2, handle_call/3, handle_cast/2, handle_info/2,code_change/3,stop/0]). +-export([init/1, terminate/2, handle_call/3, handle_cast/2, handle_info/2,code_change/3]). -export([start_doc_map/2, map_docs/2, stop_doc_map/1]). --export([reduce/3, rereduce/3,validate_doc_update/4]). +-export([reduce/3, rereduce/3,validate_doc_update/5]). -export([filter_docs/5]). -export([with_ddoc_proc/2, proc_prompt/2, ddoc_prompt/3, ddoc_proc_prompt/3, json_doc/1]). @@ -38,9 +38,6 @@ start_link() -> gen_server:start_link({local, couch_query_servers}, couch_query_servers, [], []). -stop() -> - exit(whereis(couch_query_servers), normal). - start_doc_map(Lang, Functions) -> Proc = get_os_process(Lang), lists:foreach(fun(FunctionSource) -> @@ -93,7 +90,7 @@ group_reductions_results(List) -> rereduce(_Lang, [], _ReducedValues) -> {ok, []}; rereduce(Lang, RedSrcs, ReducedValues) -> - Grouped = group_reductions_results(ReducedValues), + Grouped = group_reductions_results(ReducedValues), Results = lists:zipwith( fun (<<"_", _/binary>> = FunSrc, Values) -> @@ -146,15 +143,18 @@ os_rereduce(Lang, OsRedSrcs, KVs) -> builtin_reduce(_Re, [], _KVs, Acc) -> {ok, lists:reverse(Acc)}; -builtin_reduce(Re, [<<"_sum">>|BuiltinReds], KVs, Acc) -> +builtin_reduce(Re, [<<"_sum",_/binary>>|BuiltinReds], KVs, Acc) -> Sum = builtin_sum_rows(KVs), builtin_reduce(Re, BuiltinReds, KVs, [Sum|Acc]); -builtin_reduce(reduce, [<<"_count">>|BuiltinReds], KVs, Acc) -> +builtin_reduce(reduce, [<<"_count",_/binary>>|BuiltinReds], KVs, Acc) -> Count = length(KVs), builtin_reduce(reduce, BuiltinReds, KVs, [Count|Acc]); -builtin_reduce(rereduce, [<<"_count">>|BuiltinReds], KVs, Acc) -> +builtin_reduce(rereduce, [<<"_count",_/binary>>|BuiltinReds], KVs, Acc) -> Count = builtin_sum_rows(KVs), - builtin_reduce(rereduce, BuiltinReds, KVs, [Count|Acc]). + builtin_reduce(rereduce, BuiltinReds, KVs, [Count|Acc]); +builtin_reduce(Re, [<<"_stats",_/binary>>|BuiltinReds], KVs, Acc) -> + Stats = builtin_stats(Re, KVs), + builtin_reduce(Re, BuiltinReds, KVs, [Stats|Acc]). builtin_sum_rows(KVs) -> lists:foldl(fun @@ -164,12 +164,30 @@ builtin_sum_rows(KVs) -> throw({invalid_value, <<"builtin _sum function requires map values to be numbers">>}) end, 0, KVs). +builtin_stats(reduce, [[_,First]|Rest]) when is_number(First) -> + Stats = lists:foldl(fun([_K,V], {S,C,Mi,Ma,Sq}) when is_number(V) -> + {S+V, C+1, lists:min([Mi, V]), lists:max([Ma, V]), Sq+(V*V)}; + (_, _) -> + throw({invalid_value, + <<"builtin _stats function requires map values to be numbers">>}) + end, {First,1,First,First,First*First}, Rest), + {Sum, Cnt, Min, Max, Sqr} = Stats, + {[{sum,Sum}, {count,Cnt}, {min,Min}, {max,Max}, {sumsqr,Sqr}]}; + +builtin_stats(rereduce, [[_,First]|Rest]) -> + {[{sum,Sum0}, {count,Cnt0}, {min,Min0}, {max,Max0}, {sumsqr,Sqr0}]} = First, + Stats = lists:foldl(fun([_K,Red], {S,C,Mi,Ma,Sq}) -> + {[{sum,Sum}, {count,Cnt}, {min,Min}, {max,Max}, {sumsqr,Sqr}]} = Red, + {Sum+S, Cnt+C, lists:min([Min, Mi]), lists:max([Max, Ma]), Sqr+Sq} + end, {Sum0,Cnt0,Min0,Max0,Sqr0}, Rest), + {Sum, Cnt, Min, Max, Sqr} = Stats, + {[{sum,Sum}, {count,Cnt}, {min,Min}, {max,Max}, {sumsqr,Sqr}]}. % use the function stored in ddoc.validate_doc_update to test an update. -validate_doc_update(DDoc, EditDoc, DiskDoc, Ctx) -> +validate_doc_update(DDoc, EditDoc, DiskDoc, Ctx, SecObj) -> JsonEditDoc = couch_doc:to_json_obj(EditDoc, [revs]), - JsonDiskDoc = json_doc(DiskDoc), - case ddoc_prompt(DDoc, [<<"validate_doc_update">>], [JsonEditDoc, JsonDiskDoc, Ctx]) of + JsonDiskDoc = json_doc(DiskDoc), + case ddoc_prompt(DDoc, [<<"validate_doc_update">>], [JsonEditDoc, JsonDiskDoc, Ctx, SecObj]) of 1 -> ok; {[{<<"forbidden">>, Message}]} -> @@ -183,13 +201,17 @@ json_doc(Doc) -> couch_doc:to_json_obj(Doc, [revs]). filter_docs(Req, Db, DDoc, FName, Docs) -> - JsonReq = couch_httpd_external:json_req_obj(Req, Db), + JsonReq = case Req of + {json_req, JsonObj} -> + JsonObj; + #httpd{} = HttpReq -> + couch_httpd_external:json_req_obj(HttpReq, Db) + end, JsonDocs = [couch_doc:to_json_obj(Doc, [revs]) || Doc <- Docs], - JsonCtx = couch_util:json_user_ctx(Db), - [true, Passes] = ddoc_prompt(DDoc, [<<"filters">>, FName], [JsonDocs, JsonReq, JsonCtx]), + [true, Passes] = ddoc_prompt(DDoc, [<<"filters">>, FName], [JsonDocs, JsonReq]), {ok, Passes}. -ddoc_proc_prompt({Proc, DDocId}, FunPath, Args) -> +ddoc_proc_prompt({Proc, DDocId}, FunPath, Args) -> proc_prompt(Proc, [<<"ddoc">>, DDocId, FunPath, Args]). ddoc_prompt(DDoc, FunPath, Args) -> @@ -214,17 +236,18 @@ init([]) -> ok = couch_config:register( fun("query_servers" ++ _, _) -> - ?MODULE:stop() + supervisor:terminate_child(couch_secondary_services, query_servers), + supervisor:restart_child(couch_secondary_services, query_servers) end), ok = couch_config:register( fun("native_query_servers" ++ _, _) -> - ?MODULE:stop() + supervisor:terminate_child(couch_secondary_services, query_servers), + [supervisor:restart_child(couch_secondary_services, query_servers)] end), Langs = ets:new(couch_query_server_langs, [set, private]), PidProcs = ets:new(couch_query_server_pid_langs, [set, private]), LangProcs = ets:new(couch_query_server_procs, [set, private]), - InUse = ets:new(couch_query_server_used, [set, private]), % 'query_servers' specifies an OS command-line to execute. lists:foreach(fun({Lang, Command}) -> true = ets:insert(Langs, {?l2b(Lang), @@ -239,75 +262,71 @@ init([]) -> process_flag(trap_exit, true), {ok, {Langs, % Keyed by language name, value is {Mod,Func,Arg} PidProcs, % Keyed by PID, valus is a #proc record. - LangProcs, % Keyed by language name, value is a #proc record - InUse % Keyed by PID, value is #proc record. + LangProcs % Keyed by language name, value is a #proc record }}. -terminate(_Reason, _Server) -> +terminate(_Reason, {_Langs, PidProcs, _LangProcs}) -> + [couch_util:shutdown_sync(P) || {P,_} <- ets:tab2list(PidProcs)], ok. -handle_call({get_proc, #doc{body={Props}}=DDoc, DDocKey}, _From, {Langs, PidProcs, LangProcs, InUse}=Server) -> +handle_call({get_proc, #doc{body={Props}}=DDoc, DDocKey}, _From, {Langs, PidProcs, LangProcs}=Server) -> % Note to future self. Add max process limit. - Lang = proplists:get_value(<<"language">>, Props, <<"javascript">>), + Lang = couch_util:get_value(<<"language">>, Props, <<"javascript">>), case ets:lookup(LangProcs, Lang) of [{Lang, [P|Rest]}] -> % find a proc in the set that has the DDoc case proc_with_ddoc(DDoc, DDocKey, [P|Rest]) of - {ok, Proc} -> - % looks like the proc isn't getting dropped from the list. - % we need to change this to take a fun for equality checking - % so we can do a comparison on portnum - rem_from_list(LangProcs, Lang, Proc), - add_to_list(InUse, Lang, Proc), - {reply, {ok, Proc, get_query_server_config()}, Server}; - Error -> - {reply, Error, Server} - end; + {ok, Proc} -> + rem_from_list(LangProcs, Lang, Proc), + {reply, {ok, Proc, get_query_server_config()}, Server}; + Error -> + {reply, Error, Server} + end; _ -> case (catch new_process(Langs, Lang)) of {ok, Proc} -> add_value(PidProcs, Proc#proc.pid, Proc), case proc_with_ddoc(DDoc, DDocKey, [Proc]) of - {ok, Proc2} -> - rem_from_list(LangProcs, Lang, Proc), - add_to_list(InUse, Lang, Proc2), - {reply, {ok, Proc2, get_query_server_config()}, Server}; - Error -> - {reply, Error, Server} - end; + {ok, Proc2} -> + {reply, {ok, Proc2, get_query_server_config()}, Server}; + Error -> + {reply, Error, Server} + end; Error -> {reply, Error, Server} end end; -handle_call({get_proc, Lang}, _From, {Langs, PidProcs, LangProcs, InUse}=Server) -> +handle_call({get_proc, Lang}, _From, {Langs, PidProcs, LangProcs}=Server) -> % Note to future self. Add max process limit. case ets:lookup(LangProcs, Lang) of [{Lang, [Proc|_]}] -> - add_value(PidProcs, Proc#proc.pid, Proc), rem_from_list(LangProcs, Lang, Proc), - add_to_list(InUse, Lang, Proc), {reply, {ok, Proc, get_query_server_config()}, Server}; _ -> case (catch new_process(Langs, Lang)) of {ok, Proc} -> add_value(PidProcs, Proc#proc.pid, Proc), - add_to_list(InUse, Lang, Proc), {reply, {ok, Proc, get_query_server_config()}, Server}; Error -> {reply, Error, Server} end end; -handle_call({ret_proc, Proc}, _From, {_, _, LangProcs, InUse}=Server) -> +handle_call({unlink_proc, Pid}, _From, {_, PidProcs, _}=Server) -> + rem_value(PidProcs, Pid), + unlink(Pid), + {reply, ok, Server}; +handle_call({ret_proc, Proc}, _From, {_, PidProcs, LangProcs}=Server) -> % Along with max process limit, here we should check % if we're over the limit and discard when we are. + add_value(PidProcs, Proc#proc.pid, Proc), add_to_list(LangProcs, Proc#proc.lang, Proc), - rem_from_list(InUse, Proc#proc.lang, Proc), + link(Proc#proc.pid), {reply, true, Server}. handle_cast(_Whatever, Server) -> {noreply, Server}. -handle_info({'EXIT', Pid, Status}, {_, PidProcs, LangProcs, InUse}=Server) -> +handle_info({'EXIT', Pid, Status}, {_, PidProcs, LangProcs}=Server) -> case ets:lookup(PidProcs, Pid) of [{Pid, Proc}] -> case Status of @@ -316,11 +335,14 @@ handle_info({'EXIT', Pid, Status}, {_, PidProcs, LangProcs, InUse}=Server) -> end, rem_value(PidProcs, Pid), catch rem_from_list(LangProcs, Proc#proc.lang, Proc), - catch rem_from_list(InUse, Proc#proc.lang, Proc), {noreply, Server}; [] -> - ?LOG_DEBUG("Unknown linked process died: ~p (reason: ~p)", [Pid, Status]), - {stop, Status, Server} + case Status of + normal -> + {noreply, Server}; + _ -> + {stop, Status, Server} + end end. code_change(_OldVsn, State, _Extra) -> @@ -348,7 +370,7 @@ new_process(Langs, Lang) -> end. proc_with_ddoc(DDoc, DDocKey, LangProcs) -> - DDocProcs = lists:filter(fun(#proc{ddoc_keys=Keys}) -> + DDocProcs = lists:filter(fun(#proc{ddoc_keys=Keys}) -> lists:any(fun(Key) -> Key == DDocKey end, Keys) @@ -397,6 +419,7 @@ get_ddoc_process(#doc{} = DDoc, DDocKey) -> proc_set_timeout(Proc, list_to_integer(couch_config:get( "couchdb", "os_process_timeout", "5000"))), link(Proc#proc.pid), + gen_server:call(couch_query_servers, {unlink_proc, Proc#proc.pid}), Proc; _ -> catch proc_stop(Proc), @@ -406,11 +429,6 @@ get_ddoc_process(#doc{} = DDoc, DDocKey) -> throw(Error) end. -ret_ddoc_process(Proc) -> - true = gen_server:call(couch_query_servers, {ret_proc, Proc}), - catch unlink(Proc#proc.pid), - ok. - get_os_process(Lang) -> case gen_server:call(couch_query_servers, {get_proc, Lang}) of {ok, Proc, QueryConfig} -> @@ -419,6 +437,7 @@ get_os_process(Lang) -> proc_set_timeout(Proc, list_to_integer(couch_config:get( "couchdb", "os_process_timeout", "5000"))), link(Proc#proc.pid), + gen_server:call(couch_query_servers, {unlink_proc, Proc#proc.pid}), Proc; _ -> catch proc_stop(Proc), diff --git a/src/couchdb/couch_ref_counter.erl b/src/couchdb/couch_ref_counter.erl index 5c8e7437..5a111ab6 100644 --- a/src/couchdb/couch_ref_counter.erl +++ b/src/couchdb/couch_ref_counter.erl @@ -24,7 +24,7 @@ drop(RefCounterPid) -> drop(RefCounterPid, self()). drop(RefCounterPid, Pid) -> - gen_server:cast(RefCounterPid, {drop, Pid}). + gen_server:call(RefCounterPid, {drop, Pid}). add(RefCounterPid) -> @@ -40,17 +40,18 @@ count(RefCounterPid) -> -record(srv, { - referrers=dict:new() % a dict of each ref counting proc. + referrers=dict:new(), % a dict of each ref counting proc. + child_procs=[] }). init({Pid, ChildProcs}) -> [link(ChildProc) || ChildProc <- ChildProcs], Referrers = dict:from_list([{Pid, {erlang:monitor(process, Pid), 1}}]), - {ok, #srv{referrers=Referrers}}. + {ok, #srv{referrers=Referrers, child_procs=ChildProcs}}. -terminate(Reason, _Srv) -> - couch_util:terminate_linked(Reason), +terminate(_Reason, #srv{child_procs=ChildProcs}) -> + [couch_util:shutdown_sync(Pid) || Pid <- ChildProcs], ok. @@ -65,10 +66,8 @@ handle_call({add, Pid},_From, #srv{referrers=Referrers}=Srv) -> {reply, ok, Srv#srv{referrers=Referrers2}}; handle_call(count, _From, Srv) -> {monitors, Monitors} = process_info(self(), monitors), - {reply, length(Monitors), Srv}. - - -handle_cast({drop, Pid}, #srv{referrers=Referrers}=Srv) -> + {reply, length(Monitors), Srv}; +handle_call({drop, Pid}, _From, #srv{referrers=Referrers}=Srv) -> Referrers2 = case dict:find(Pid, Referrers) of {ok, {MonRef, 1}} -> @@ -79,7 +78,16 @@ handle_cast({drop, Pid}, #srv{referrers=Referrers}=Srv) -> error -> Referrers end, - maybe_close_async(Srv#srv{referrers=Referrers2}). + Srv2 = Srv#srv{referrers=Referrers2}, + case should_close() of + true -> + {stop,normal,ok,Srv2}; + false -> + {reply, ok, Srv2} + end. + +handle_cast(Msg, _Srv)-> + exit({unknown_msg,Msg}). code_change(_OldVsn, State, _Extra) -> @@ -87,21 +95,17 @@ code_change(_OldVsn, State, _Extra) -> handle_info({'DOWN', MonRef, _, Pid, _}, #srv{referrers=Referrers}=Srv) -> {ok, {MonRef, _RefCount}} = dict:find(Pid, Referrers), - maybe_close_async(Srv#srv{referrers=dict:erase(Pid, Referrers)}). + Srv2 = Srv#srv{referrers=dict:erase(Pid, Referrers)}, + case should_close() of + true -> + {stop,normal,Srv2}; + false -> + {noreply,Srv2} + end. should_close() -> case process_info(self(), monitors) of - {monitors, []} -> - true; - _ -> - false - end. - -maybe_close_async(Srv) -> - case should_close() of - true -> - {stop,normal,Srv}; - false -> - {noreply,Srv} + {monitors, []} -> true; + _ -> false end. diff --git a/src/couchdb/couch_rep.erl b/src/couchdb/couch_rep.erl index 98413f23..c804b49d 100644 --- a/src/couchdb/couch_rep.erl +++ b/src/couchdb/couch_rep.erl @@ -12,7 +12,7 @@ -module(couch_rep). -behaviour(gen_server). --export([init/1, handle_call/3, handle_cast/2, handle_info/2, terminate/2, +-export([init/1, handle_call/3, handle_cast/2, handle_info/2, terminate/2, code_change/3]). -export([replicate/2, checkpoint/1]). @@ -34,6 +34,7 @@ start_seq, history, + session_id, source_log, target_log, rep_starttime, @@ -45,7 +46,10 @@ complete = false, committed_seq = 0, - stats = nil + stats = nil, + doc_ids = nil, + source_db_update_notifier = nil, + target_db_update_notifier = nil }). %% convenience function to do a simple replication from the shell @@ -67,13 +71,24 @@ replicate({Props}=PostBody, UserCtx) -> [?MODULE] }, - Server = start_replication_server(Replicator), - - case proplists:get_value(<<"continuous">>, Props, false) of + case couch_util:get_value(<<"cancel">>, Props, false) of true -> - {ok, {continuous, ?l2b(BaseId)}}; + case supervisor:terminate_child(couch_rep_sup, BaseId ++ Extension) of + {error, not_found} -> + {error, not_found}; + ok -> + ok = supervisor:delete_child(couch_rep_sup, BaseId ++ Extension), + {ok, {cancelled, ?l2b(BaseId)}} + end; false -> - get_result(Server, PostBody, UserCtx) + Server = start_replication_server(Replicator), + + case couch_util:get_value(<<"continuous">>, Props, false) of + true -> + {ok, {continuous, ?l2b(BaseId)}}; + false -> + get_result(Server, PostBody, UserCtx) + end end. checkpoint(Server) -> @@ -93,35 +108,64 @@ get_result(Server, PostBody, UserCtx) -> end. init(InitArgs) -> - try do_init(InitArgs) - catch throw:{db_not_found, DbUrl} -> {stop, {db_not_found, DbUrl}} end. + try + do_init(InitArgs) + catch + throw:Error -> + {stop, Error} + end. do_init([RepId, {PostProps}, UserCtx] = InitArgs) -> process_flag(trap_exit, true), - SourceProps = proplists:get_value(<<"source">>, PostProps), - TargetProps = proplists:get_value(<<"target">>, PostProps), + SourceProps = couch_util:get_value(<<"source">>, PostProps), + TargetProps = couch_util:get_value(<<"target">>, PostProps), - Continuous = proplists:get_value(<<"continuous">>, PostProps, false), - CreateTarget = proplists:get_value(<<"create_target">>, PostProps, false), + DocIds = couch_util:get_value(<<"doc_ids">>, PostProps, nil), + Continuous = couch_util:get_value(<<"continuous">>, PostProps, false), + CreateTarget = couch_util:get_value(<<"create_target">>, PostProps, false), - Source = open_db(SourceProps, UserCtx), - Target = open_db(TargetProps, UserCtx, CreateTarget), - - SourceLog = open_replication_log(Source, RepId), - TargetLog = open_replication_log(Target, RepId), + ProxyParams = parse_proxy_params( + couch_util:get_value(<<"proxy">>, PostProps, [])), + Source = open_db(SourceProps, UserCtx, ProxyParams), + Target = open_db(TargetProps, UserCtx, ProxyParams, CreateTarget), SourceInfo = dbinfo(Source), TargetInfo = dbinfo(Target), + + case DocIds of + List when is_list(List) -> + % Fast replication using only a list of doc IDs to replicate. + % Replication sessions, checkpoints and logs are not created + % since the update sequence number of the source DB is not used + % for determining which documents are copied into the target DB. + SourceLog = nil, + TargetLog = nil, + + StartSeq = nil, + History = nil, + + ChangesFeed = nil, + MissingRevs = nil, + + {ok, Reader} = + couch_rep_reader:start_link(self(), Source, DocIds, PostProps); + + _ -> + % Replication using the _changes API (DB sequence update numbers). + SourceLog = open_replication_log(Source, RepId), + TargetLog = open_replication_log(Target, RepId), - {StartSeq, History} = compare_replication_logs(SourceLog, TargetLog), - - {ok, ChangesFeed} = - couch_rep_changes_feed:start_link(self(), Source, StartSeq, PostProps), - {ok, MissingRevs} = - couch_rep_missing_revs:start_link(self(), Target, ChangesFeed, PostProps), - {ok, Reader} = - couch_rep_reader:start_link(self(), Source, MissingRevs, PostProps), + {StartSeq, History} = compare_replication_logs(SourceLog, TargetLog), + + {ok, ChangesFeed} = + couch_rep_changes_feed:start_link(self(), Source, StartSeq, PostProps), + {ok, MissingRevs} = + couch_rep_missing_revs:start_link(self(), Target, ChangesFeed, PostProps), + {ok, Reader} = + couch_rep_reader:start_link(self(), Source, MissingRevs, PostProps) + end, + {ok, Writer} = couch_rep_writer:start_link(self(), Target, Reader, PostProps), @@ -152,11 +196,15 @@ do_init([RepId, {PostProps}, UserCtx] = InitArgs) -> start_seq = StartSeq, history = History, + session_id = couch_uuids:random(), source_log = SourceLog, target_log = TargetLog, rep_starttime = httpd_util:rfc1123_date(), - src_starttime = proplists:get_value(instance_start_time, SourceInfo), - tgt_starttime = proplists:get_value(instance_start_time, TargetInfo) + src_starttime = couch_util:get_value(instance_start_time, SourceInfo), + tgt_starttime = couch_util:get_value(instance_start_time, TargetInfo), + doc_ids = DocIds, + source_db_update_notifier = source_db_update_notifier(Source), + target_db_update_notifier = target_db_update_notifier(Target) }, {ok, State}. @@ -164,7 +212,21 @@ handle_call(get_result, From, #state{complete=true, listeners=[]} = State) -> {stop, normal, State#state{listeners=[From]}}; handle_call(get_result, From, State) -> Listeners = State#state.listeners, - {noreply, State#state{listeners=[From|Listeners]}}. + {noreply, State#state{listeners=[From|Listeners]}}; + +handle_call(get_source_db, _From, #state{source = Source} = State) -> + {reply, {ok, Source}, State}; + +handle_call(get_target_db, _From, #state{target = Target} = State) -> + {reply, {ok, Target}, State}. + +handle_cast(reopen_source_db, #state{source = Source} = State) -> + {ok, NewSource} = couch_db:reopen(Source), + {noreply, State#state{source = NewSource}}; + +handle_cast(reopen_target_db, #state{target = Target} = State) -> + {ok, NewTarget} = couch_db:reopen(Target), + {noreply, State#state{target = NewTarget}}; handle_cast(do_checkpoint, State) -> {noreply, do_checkpoint(State)}; @@ -254,29 +316,37 @@ start_replication_server(Replicator) -> {error, {already_started, Pid}} = supervisor:start_child(couch_rep_sup, Replicator), ?LOG_DEBUG("replication ~p already running at ~p", [RepId, Pid]), - Pid + Pid; + {error, {db_not_found, DbUrl}} -> + throw({db_not_found, <<"could not open ", DbUrl/binary>>}); + {error, {unauthorized, DbUrl}} -> + throw({unauthorized, + <<"unauthorized to access database ", DbUrl/binary>>}) end; {error, {already_started, Pid}} -> ?LOG_DEBUG("replication ~p already running at ~p", [RepId, Pid]), Pid; {error, {{db_not_found, DbUrl}, _}} -> - throw({db_not_found, <<"could not open ", DbUrl/binary>>}) + throw({db_not_found, <<"could not open ", DbUrl/binary>>}); + {error, {{unauthorized, DbUrl}, _}} -> + throw({unauthorized, + <<"unauthorized to access database ", DbUrl/binary>>}) end. compare_replication_logs(SrcDoc, TgtDoc) -> #doc{body={RepRecProps}} = SrcDoc, #doc{body={RepRecPropsTgt}} = TgtDoc, - case proplists:get_value(<<"session_id">>, RepRecProps) == - proplists:get_value(<<"session_id">>, RepRecPropsTgt) of + case couch_util:get_value(<<"session_id">>, RepRecProps) == + couch_util:get_value(<<"session_id">>, RepRecPropsTgt) of true -> % if the records have the same session id, % then we have a valid replication history - OldSeqNum = proplists:get_value(<<"source_last_seq">>, RepRecProps, 0), - OldHistory = proplists:get_value(<<"history">>, RepRecProps, []), + OldSeqNum = couch_util:get_value(<<"source_last_seq">>, RepRecProps, 0), + OldHistory = couch_util:get_value(<<"history">>, RepRecProps, []), {OldSeqNum, OldHistory}; false -> - SourceHistory = proplists:get_value(<<"history">>, RepRecProps, []), - TargetHistory = proplists:get_value(<<"history">>, RepRecPropsTgt, []), + SourceHistory = couch_util:get_value(<<"history">>, RepRecProps, []), + TargetHistory = couch_util:get_value(<<"history">>, RepRecPropsTgt, []), ?LOG_INFO("Replication records differ. " "Scanning histories to find a common ancestor.", []), ?LOG_DEBUG("Record on source:~p~nRecord on target:~p~n", @@ -288,18 +358,18 @@ compare_rep_history(S, T) when S =:= [] orelse T =:= [] -> ?LOG_INFO("no common ancestry -- performing full replication", []), {0, []}; compare_rep_history([{S}|SourceRest], [{T}|TargetRest]=Target) -> - SourceId = proplists:get_value(<<"session_id">>, S), + SourceId = couch_util:get_value(<<"session_id">>, S), case has_session_id(SourceId, Target) of true -> - RecordSeqNum = proplists:get_value(<<"recorded_seq">>, S, 0), + RecordSeqNum = couch_util:get_value(<<"recorded_seq">>, S, 0), ?LOG_INFO("found a common replication record with source_seq ~p", [RecordSeqNum]), {RecordSeqNum, SourceRest}; false -> - TargetId = proplists:get_value(<<"session_id">>, T), + TargetId = couch_util:get_value(<<"session_id">>, T), case has_session_id(TargetId, SourceRest) of true -> - RecordSeqNum = proplists:get_value(<<"recorded_seq">>, T, 0), + RecordSeqNum = couch_util:get_value(<<"recorded_seq">>, T, 0), ?LOG_INFO("found a common replication record with source_seq ~p", [RecordSeqNum]), {RecordSeqNum, TargetRest}; @@ -314,31 +384,45 @@ close_db(Db) -> couch_db:close(Db). dbname(#http_db{url = Url}) -> - Url; + couch_util:url_strip_password(Url); dbname(#db{name = Name}) -> Name. dbinfo(#http_db{} = Db) -> {DbProps} = couch_rep_httpc:request(Db), - [{list_to_atom(?b2l(K)), V} || {K,V} <- DbProps]; + [{list_to_existing_atom(?b2l(K)), V} || {K,V} <- DbProps]; dbinfo(Db) -> {ok, Info} = couch_db:get_db_info(Db), Info. +do_terminate(#state{doc_ids=DocIds} = State) when is_list(DocIds) -> + #state{ + listeners = Listeners, + rep_starttime = ReplicationStartTime, + stats = Stats + } = State, + + RepByDocsJson = {[ + {<<"start_time">>, ?l2b(ReplicationStartTime)}, + {<<"end_time">>, ?l2b(httpd_util:rfc1123_date())}, + {<<"docs_read">>, ets:lookup_element(Stats, docs_read, 2)}, + {<<"docs_written">>, ets:lookup_element(Stats, docs_written, 2)}, + {<<"doc_write_failures">>, + ets:lookup_element(Stats, doc_write_failures, 2)} + ]}, + + terminate_cleanup(State), + [gen_server:reply(L, {ok, RepByDocsJson}) || L <- lists:reverse(Listeners)]; + do_terminate(State) -> #state{ checkpoint_history = CheckpointHistory, committed_seq = NewSeq, listeners = Listeners, source = Source, - target = Target, continuous = Continuous, - stats = Stats, source_log = #doc{body={OldHistory}} } = State, - couch_task_status:update("Finishing"), - ets:delete(Stats), - close_db(Target), NewRepHistory = case CheckpointHistory of nil -> @@ -366,12 +450,25 @@ do_terminate(State) -> false -> [gen_server:reply(R, retry) || R <- OtherListeners] end, - close_db(Source). + couch_task_status:update("Finishing"), + terminate_cleanup(State). + +terminate_cleanup(State) -> + close_db(State#state.source), + close_db(State#state.target), + stop_db_update_notifier(State#state.source_db_update_notifier), + stop_db_update_notifier(State#state.target_db_update_notifier), + ets:delete(State#state.stats). + +stop_db_update_notifier(nil) -> + ok; +stop_db_update_notifier(Notifier) -> + couch_db_update_notifier:stop(Notifier). has_session_id(_SessionId, []) -> false; has_session_id(SessionId, [{Props} | Rest]) -> - case proplists:get_value(<<"session_id">>, Props, nil) of + case couch_util:get_value(<<"session_id">>, Props, nil) of SessionId -> true; _Else -> @@ -380,8 +477,8 @@ has_session_id(SessionId, [{Props} | Rest]) -> maybe_append_options(Options, Props) -> lists:foldl(fun(Option, Acc) -> - Acc ++ - case proplists:get_value(Option, Props, false) of + Acc ++ + case couch_util:get_value(Option, Props, false) of true -> "+" ++ ?b2l(Option); false -> @@ -393,21 +490,32 @@ make_replication_id({Props}, UserCtx) -> %% funky algorithm to preserve backwards compatibility {ok, HostName} = inet:gethostname(), % Port = mochiweb_socket_server:get(couch_httpd, port), - Src = get_rep_endpoint(UserCtx, proplists:get_value(<<"source">>, Props)), - Tgt = get_rep_endpoint(UserCtx, proplists:get_value(<<"target">>, Props)), - Base = couch_util:to_hex(erlang:md5(term_to_binary([HostName, Src, Tgt]))), + Src = get_rep_endpoint(UserCtx, couch_util:get_value(<<"source">>, Props)), + Tgt = get_rep_endpoint(UserCtx, couch_util:get_value(<<"target">>, Props)), + Base = [HostName, Src, Tgt] ++ + case couch_util:get_value(<<"filter">>, Props) of + undefined -> + case couch_util:get_value(<<"doc_ids">>, Props) of + undefined -> + []; + DocIds -> + [DocIds] + end; + Filter -> + [Filter, couch_util:get_value(<<"query_params">>, Props, {[]})] + end, Extension = maybe_append_options( [<<"continuous">>, <<"create_target">>], Props), - {Base, Extension}. + {couch_util:to_hex(couch_util:md5(term_to_binary(Base))), Extension}. maybe_add_trailing_slash(Url) -> re:replace(Url, "[^/]$", "&/", [{return, list}]). get_rep_endpoint(_UserCtx, {Props}) -> - Url = maybe_add_trailing_slash(proplists:get_value(<<"url">>, Props)), - {BinHeaders} = proplists:get_value(<<"headers">>, Props, {[]}), - {Auth} = proplists:get_value(<<"auth">>, Props, {[]}), - case proplists:get_value(<<"oauth">>, Auth) of + Url = maybe_add_trailing_slash(couch_util:get_value(<<"url">>, Props)), + {BinHeaders} = couch_util:get_value(<<"headers">>, Props, {[]}), + {Auth} = couch_util:get_value(<<"auth">>, Props, {[]}), + case couch_util:get_value(<<"oauth">>, Auth) of undefined -> {remote, Url, [{?b2l(K),?b2l(V)} || {K,V} <- BinHeaders]}; {OAuth} -> @@ -442,38 +550,48 @@ open_replication_log(Db, RepId) -> #doc{id=DocId} end. -open_db(Props, UserCtx) -> - open_db(Props, UserCtx, false). +open_db(Props, UserCtx, ProxyParams) -> + open_db(Props, UserCtx, ProxyParams, false). -open_db({Props}, _UserCtx, CreateTarget) -> - Url = maybe_add_trailing_slash(proplists:get_value(<<"url">>, Props)), - {AuthProps} = proplists:get_value(<<"auth">>, Props, {[]}), - {BinHeaders} = proplists:get_value(<<"headers">>, Props, {[]}), +open_db({Props}, _UserCtx, ProxyParams, CreateTarget) -> + Url = maybe_add_trailing_slash(couch_util:get_value(<<"url">>, Props)), + {AuthProps} = couch_util:get_value(<<"auth">>, Props, {[]}), + {BinHeaders} = couch_util:get_value(<<"headers">>, Props, {[]}), Headers = [{?b2l(K),?b2l(V)} || {K,V} <- BinHeaders], DefaultHeaders = (#http_db{})#http_db.headers, - Db = #http_db{ + Db1 = #http_db{ url = Url, auth = AuthProps, headers = lists:ukeymerge(1, Headers, DefaultHeaders) }, + Db = Db1#http_db{ + options = Db1#http_db.options ++ ProxyParams ++ + couch_rep_httpc:ssl_options(Db1) + }, couch_rep_httpc:db_exists(Db, CreateTarget); -open_db(<<"http://",_/binary>>=Url, _, CreateTarget) -> - open_db({[{<<"url">>,Url}]}, [], CreateTarget); -open_db(<<"https://",_/binary>>=Url, _, CreateTarget) -> - open_db({[{<<"url">>,Url}]}, [], CreateTarget); -open_db(<<DbName/binary>>, UserCtx, CreateTarget) -> - case CreateTarget of - true -> - ok = couch_httpd:verify_is_server_admin(UserCtx), - couch_server:create(DbName, [{user_ctx, UserCtx}]); - false -> ok - end, - - case couch_db:open(DbName, [{user_ctx, UserCtx}]) of - {ok, Db} -> - couch_db:monitor(Db), - Db; - {not_found, no_db_file} -> throw({db_not_found, DbName}) +open_db(<<"http://",_/binary>>=Url, _, ProxyParams, CreateTarget) -> + open_db({[{<<"url">>,Url}]}, [], ProxyParams, CreateTarget); +open_db(<<"https://",_/binary>>=Url, _, ProxyParams, CreateTarget) -> + open_db({[{<<"url">>,Url}]}, [], ProxyParams, CreateTarget); +open_db(<<DbName/binary>>, UserCtx, _ProxyParams, CreateTarget) -> + try + case CreateTarget of + true -> + ok = couch_httpd:verify_is_server_admin(UserCtx), + couch_server:create(DbName, [{user_ctx, UserCtx}]); + false -> + ok + end, + + case couch_db:open(DbName, [{user_ctx, UserCtx}]) of + {ok, Db} -> + couch_db:monitor(Db), + Db; + {not_found, no_db_file} -> + throw({db_not_found, DbName}) + end + catch throw:{unauthorized, _} -> + throw({unauthorized, DbName}) end. schedule_checkpoint(#state{checkpoint_scheduled = nil} = State) -> @@ -495,6 +613,7 @@ do_checkpoint(State) -> committed_seq = NewSeqNum, start_seq = StartSeqNum, history = OldHistory, + session_id = SessionId, source_log = SourceLog, target_log = TargetLog, rep_starttime = ReplicationStartTime, @@ -506,7 +625,6 @@ do_checkpoint(State) -> {SrcInstanceStartTime, TgtInstanceStartTime} -> ?LOG_INFO("recording a checkpoint for ~s -> ~s at source update_seq ~p", [dbname(Source), dbname(Target), NewSeqNum]), - SessionId = couch_uuids:random(), NewHistoryEntry = {[ {<<"session_id">>, SessionId}, {<<"start_time">>, list_to_binary(ReplicationStartTime)}, @@ -518,7 +636,7 @@ do_checkpoint(State) -> {<<"missing_found">>, ets:lookup_element(Stats, missing_revs, 2)}, {<<"docs_read">>, ets:lookup_element(Stats, docs_read, 2)}, {<<"docs_written">>, ets:lookup_element(Stats, docs_written, 2)}, - {<<"doc_write_failures">>, + {<<"doc_write_failures">>, ets:lookup_element(Stats, doc_write_failures, 2)} ]}, % limit history to 50 entries @@ -559,7 +677,7 @@ do_checkpoint(State) -> close_db(Target), close_db(Source), {ok, NewState} = init(State#state.init_args), - NewState + NewState#state{listeners=State#state.listeners} end. commit_to_both(Source, Target, RequiredSeq) -> @@ -580,17 +698,22 @@ commit_to_both(Source, Target, RequiredSeq) -> end, {SourceStartTime, TargetStartTime}. -ensure_full_commit(#http_db{} = Target) -> +ensure_full_commit(#http_db{headers = Headers} = Target) -> + Headers1 = [ + {"Content-Length", 0} | + couch_util:proplist_apply_field( + {"Content-Type", "application/json"}, Headers) + ], Req = Target#http_db{ resource = "_ensure_full_commit", method = post, - body = true + headers = Headers1 }, {ResultProps} = couch_rep_httpc:request(Req), - true = proplists:get_value(<<"ok">>, ResultProps), - proplists:get_value(<<"instance_start_time">>, ResultProps); + true = couch_util:get_value(<<"ok">>, ResultProps), + couch_util:get_value(<<"instance_start_time">>, ResultProps); ensure_full_commit(Target) -> - {ok, NewDb} = couch_db:open(Target#db.name, []), + {ok, NewDb} = couch_db:open_int(Target#db.name, []), UpdateSeq = couch_db:get_update_seq(Target), CommitSeq = couch_db:get_committed_update_seq(NewDb), InstanceStartTime = NewDb#db.instance_start_time, @@ -605,20 +728,25 @@ ensure_full_commit(Target) -> InstanceStartTime end. -ensure_full_commit(#http_db{} = Source, RequiredSeq) -> +ensure_full_commit(#http_db{headers = Headers} = Source, RequiredSeq) -> + Headers1 = [ + {"Content-Length", 0} | + couch_util:proplist_apply_field( + {"Content-Type", "application/json"}, Headers) + ], Req = Source#http_db{ resource = "_ensure_full_commit", method = post, - body = true, - qs = [{seq, RequiredSeq}] + qs = [{seq, RequiredSeq}], + headers = Headers1 }, {ResultProps} = couch_rep_httpc:request(Req), - case proplists:get_value(<<"ok">>, ResultProps) of + case couch_util:get_value(<<"ok">>, ResultProps) of true -> - proplists:get_value(<<"instance_start_time">>, ResultProps); + couch_util:get_value(<<"instance_start_time">>, ResultProps); undefined -> nil end; ensure_full_commit(Source, RequiredSeq) -> - {ok, NewDb} = couch_db:open(Source#db.name, []), + {ok, NewDb} = couch_db:open_int(Source#db.name, []), CommitSeq = couch_db:get_committed_update_seq(NewDb), InstanceStartTime = NewDb#db.instance_start_time, couch_db:close(NewDb), @@ -640,7 +768,7 @@ update_local_doc(#http_db{} = Db, #doc{id=DocId} = Doc) -> headers = [{"x-couch-full-commit", "false"} | Db#http_db.headers] }, {ResponseMembers} = couch_rep_httpc:request(Req), - Rev = proplists:get_value(<<"rev">>, ResponseMembers), + Rev = couch_util:get_value(<<"rev">>, ResponseMembers), couch_doc:parse_rev(Rev); update_local_doc(Db, Doc) -> {ok, Result} = couch_db:update_doc(Db, Doc, [delay_commit]), @@ -649,7 +777,46 @@ update_local_doc(Db, Doc) -> up_to_date(#http_db{}, _Seq) -> true; up_to_date(Source, Seq) -> - {ok, NewDb} = couch_db:open(Source#db.name, []), + {ok, NewDb} = couch_db:open_int(Source#db.name, []), T = NewDb#db.update_seq == Seq, couch_db:close(NewDb), T. + +parse_proxy_params(ProxyUrl) when is_binary(ProxyUrl) -> + parse_proxy_params(?b2l(ProxyUrl)); +parse_proxy_params([]) -> + []; +parse_proxy_params(ProxyUrl) -> + {url, _, Base, Port, User, Passwd, _Path, _Proto} = + ibrowse_lib:parse_url(ProxyUrl), + [{proxy_host, Base}, {proxy_port, Port}] ++ + case is_list(User) andalso is_list(Passwd) of + false -> + []; + true -> + [{proxy_user, User}, {proxy_password, Passwd}] + end. + +source_db_update_notifier(#db{name = DbName}) -> + Server = self(), + {ok, Notifier} = couch_db_update_notifier:start_link( + fun({compacted, DbName1}) when DbName1 =:= DbName -> + ok = gen_server:cast(Server, reopen_source_db); + (_) -> + ok + end), + Notifier; +source_db_update_notifier(_) -> + nil. + +target_db_update_notifier(#db{name = DbName}) -> + Server = self(), + {ok, Notifier} = couch_db_update_notifier:start_link( + fun({compacted, DbName1}) when DbName1 =:= DbName -> + ok = gen_server:cast(Server, reopen_target_db); + (_) -> + ok + end), + Notifier; +target_db_update_notifier(_) -> + nil. diff --git a/src/couchdb/couch_rep_att.erl b/src/couchdb/couch_rep_att.erl index 6b576a01..3c09b90a 100644 --- a/src/couchdb/couch_rep_att.erl +++ b/src/couchdb/couch_rep_att.erl @@ -29,11 +29,13 @@ convert_stub(#att{data=stub, name=Name} = Attachment, Attachment#att{data=RcvFun}. cleanup() -> - receive + receive {ibrowse_async_response, _, _} -> %% TODO maybe log, didn't expect to have data here cleanup(); - {ibrowse_async_response_end, _} -> + {ibrowse_async_response_end, _} -> + cleanup(); + {ibrowse_async_headers, _, _, _} -> cleanup() after 0 -> erase(), @@ -43,13 +45,27 @@ cleanup() -> % internal funs attachment_receiver(Ref, Request) -> - case get(Ref) of + try case get(Ref) of undefined -> {ReqId, ContentEncoding} = start_http_request(Request), put(Ref, {ReqId, ContentEncoding}), receive_data(Ref, ReqId, ContentEncoding); {ReqId, ContentEncoding} -> receive_data(Ref, ReqId, ContentEncoding) + end + catch + throw:{attachment_request_failed, _} -> + case {Request#http_db.retries, Request#http_db.pause} of + {0, _} -> + ?LOG_INFO("request for ~p failed", [Request#http_db.resource]), + throw({attachment_request_failed, max_retries_reached}); + {N, Pause} when N > 0 -> + ?LOG_INFO("request for ~p timed out, retrying in ~p seconds", + [Request#http_db.resource, Pause/1000]), + timer:sleep(Pause), + cleanup(), + attachment_receiver(Ref, Request#http_db{retries = N-1}) + end end. receive_data(Ref, ReqId, ContentEncoding) -> @@ -63,14 +79,12 @@ receive_data(Ref, ReqId, ContentEncoding) -> throw({attachment_request_failed, Err}); {ibrowse_async_response, ReqId, Data} -> % ?LOG_DEBUG("got ~p bytes for ~p", [size(Data), ReqId]), - if ContentEncoding =:= "gzip" -> - zlib:gunzip(Data); - true -> - Data - end; + Data; {ibrowse_async_response_end, ReqId} -> ?LOG_ERROR("streaming att. ended but more data requested ~p", [ReqId]), throw({attachment_request_failed, premature_end}) + after 31000 -> + throw({attachment_request_failed, timeout}) end. start_http_request(Req) -> @@ -84,13 +98,15 @@ start_http_request(Req) -> {ok, ContentEncoding, NewReqId} -> {NewReqId, ContentEncoding} end + after 10000 -> + throw({attachment_request_failed, timeout}) end. validate_headers(_Req, 200, Headers) -> MochiHeaders = mochiweb_headers:make(Headers), {ok, mochiweb_headers:get_value("Content-Encoding", MochiHeaders)}; validate_headers(Req, Code, Headers) when Code > 299, Code < 400 -> - Url = mochiweb_headers:get_value("Location",mochiweb_headers:make(Headers)), + Url = couch_rep_httpc:redirect_url(Headers, Req#http_db.url), NewReq = couch_rep_httpc:redirected_request(Req, Url), {ibrowse_req_id, ReqId} = couch_rep_httpc:request(NewReq), receive {ibrowse_async_headers, ReqId, NewCode, NewHeaders} -> diff --git a/src/couchdb/couch_rep_changes_feed.erl b/src/couchdb/couch_rep_changes_feed.erl index cdfed6a0..7f7d3a38 100644 --- a/src/couchdb/couch_rep_changes_feed.erl +++ b/src/couchdb/couch_rep_changes_feed.erl @@ -12,7 +12,7 @@ -module(couch_rep_changes_feed). -behaviour(gen_server). --export([init/1, handle_call/3, handle_cast/2, handle_info/2, terminate/2, +-export([init/1, handle_call/3, handle_cast/2, handle_info/2, terminate/2, code_change/3]). -export([start_link/4, next/1, stop/1]). @@ -43,44 +43,73 @@ next(Server) -> gen_server:call(Server, next_changes, infinity). stop(Server) -> - gen_server:call(Server, stop). + catch gen_server:call(Server, stop), + ok. -init([_Parent, #http_db{}=Source, Since, PostProps] = Args) -> +init([Parent, #http_db{}=Source, Since, PostProps]) -> process_flag(trap_exit, true), - Feed = case proplists:get_value(<<"continuous">>, PostProps, false) of + Feed = case couch_util:get_value(<<"continuous">>, PostProps, false) of false -> normal; true -> continuous end, + BaseQS = [ + {"style", all_docs}, + {"heartbeat", 10000}, + {"since", Since}, + {"feed", Feed} + ], + QS = case couch_util:get_value(<<"filter">>, PostProps) of + undefined -> + BaseQS; + FilterName -> + {Params} = couch_util:get_value(<<"query_params">>, PostProps, {[]}), + lists:foldr( + fun({K, V}, QSAcc) -> + Ks = couch_util:to_list(K), + case proplists:is_defined(Ks, QSAcc) of + true -> + QSAcc; + false -> + [{Ks, V} | QSAcc] + end + end, + [{"filter", FilterName} | BaseQS], + Params + ) + end, Pid = couch_rep_httpc:spawn_link_worker_process(Source), Req = Source#http_db{ resource = "_changes", - qs = [{style, all_docs}, {heartbeat, 10000}, {since, Since}, - {feed, Feed}], + qs = QS, conn = Pid, - options = [{stream_to, {self(), once}}, {response_format, binary}, - {inactivity_timeout, 31000}], % miss 3 heartbeats, assume death + options = [{stream_to, {self(), once}}] ++ + lists:keydelete(inactivity_timeout, 1, Source#http_db.options), headers = Source#http_db.headers -- [{"Accept-Encoding", "gzip"}] }, {ibrowse_req_id, ReqId} = couch_rep_httpc:request(Req), + Args = [Parent, Req, Since, PostProps], receive {ibrowse_async_headers, ReqId, "200", _} -> ibrowse:stream_next(ReqId), {ok, #state{conn=Pid, last_seq=Since, reqid=ReqId, init_args=Args}}; {ibrowse_async_headers, ReqId, Code, Hdrs} when Code=="301"; Code=="302" -> - catch ibrowse:stop_worker_process(Pid), - Url2 = mochiweb_headers:get_value("Location", mochiweb_headers:make(Hdrs)), - %% TODO use couch_httpc:request instead of start_http_request - {Pid2, ReqId2} = start_http_request(Url2), + stop_link_worker(Pid), + Url2 = couch_rep_httpc:redirect_url(Hdrs, Req#http_db.url), + Req2 = couch_rep_httpc:redirected_request(Req, Url2), + Pid2 = couch_rep_httpc:spawn_link_worker_process(Req2), + Req3 = Req2#http_db{conn = Pid2}, + {ibrowse_req_id, ReqId2} = couch_rep_httpc:request(Req3), + Args2 = [Parent, Req3, Since, PostProps], receive {ibrowse_async_headers, ReqId2, "200", _} -> - {ok, #state{conn=Pid2, last_seq=Since, reqid=ReqId2, init_args=Args}} + {ok, #state{conn=Pid2, last_seq=Since, reqid=ReqId2, init_args=Args2}} after 30000 -> {stop, changes_timeout} end; {ibrowse_async_headers, ReqId, "404", _} -> - catch ibrowse:stop_worker_process(Pid), + stop_link_worker(Pid), ?LOG_INFO("source doesn't have _changes, trying _all_docs_by_seq", []), Self = self(), BySeqPid = spawn_link(fun() -> by_seq_loop(Self, Source, Since) end), @@ -94,20 +123,55 @@ init([_Parent, #http_db{}=Source, Since, PostProps] = Args) -> init([_Parent, Source, Since, PostProps] = InitArgs) -> process_flag(trap_exit, true), Server = self(), - ChangesPid = - case proplists:get_value(<<"continuous">>, PostProps, false) of - false -> - spawn_link(fun() -> send_local_changes_once(Server, Source, Since) end); - true -> - spawn_link(fun() -> - Self = self(), - {ok, _} = couch_db_update_notifier:start_link(fun(Msg) -> - local_update_notification(Self, Source#db.name, Msg) end), - send_local_changes_forever(Server, Source, Since) + ChangesArgs = #changes_args{ + style = all_docs, + since = Since, + filter = ?b2l(couch_util:get_value(<<"filter">>, PostProps, <<>>)), + feed = case couch_util:get_value(<<"continuous">>, PostProps, false) of + true -> + "continuous"; + false -> + "normal" + end, + timeout = infinity + }, + ChangesPid = spawn_link(fun() -> + ChangesFeedFun = couch_changes:handle_changes( + ChangesArgs, + {json_req, filter_json_req(Source, PostProps)}, + Source + ), + ChangesFeedFun(fun({change, Change, _}, _) -> + gen_server:call(Server, {add_change, Change}, infinity); + (_, _) -> + ok end) - end, + end), {ok, #state{changes_loop=ChangesPid, init_args=InitArgs}}. +filter_json_req(Db, PostProps) -> + case couch_util:get_value(<<"filter">>, PostProps) of + undefined -> + {[]}; + FilterName -> + {Query} = couch_util:get_value(<<"query_params">>, PostProps, {[]}), + {ok, Info} = couch_db:get_db_info(Db), + % simulate a request to db_name/_changes + {[ + {<<"info">>, {Info}}, + {<<"id">>, null}, + {<<"method">>, 'GET'}, + {<<"path">>, [couch_db:name(Db), <<"_changes">>]}, + {<<"query">>, {[{<<"filter">>, FilterName} | Query]}}, + {<<"headers">>, []}, + {<<"body">>, []}, + {<<"peer">>, <<"replicator">>}, + {<<"form">>, []}, + {<<"cookie">>, []}, + {<<"userCtx">>, couch_util:json_user_ctx(Db)} + ]} + end. + handle_call({add_change, Row}, From, State) -> handle_add_change(Row, From, State); @@ -123,7 +187,7 @@ handle_cast(_Msg, State) -> handle_info({ibrowse_async_headers, Id, Code, Hdrs}, #state{reqid=Id}=State) -> handle_headers(list_to_integer(Code), Hdrs, State); -handle_info({ibrowse_async_response, Id, {error,connection_closed}}, +handle_info({ibrowse_async_response, Id, {error, sel_conn_closed}}, #state{reqid=Id}=State) -> handle_retry(State); @@ -144,9 +208,20 @@ handle_info({'EXIT', From, Reason}, #state{changes_loop=From} = State) -> ?LOG_ERROR("changes_loop died with reason ~p", [Reason]), {stop, changes_loop_died, State}; -handle_info(Msg, State) -> - ?LOG_DEBUG("unexpected message at changes_feed ~p", [Msg]), - {noreply, State}. +handle_info({'EXIT', From, Reason}, State) -> + ?LOG_ERROR("changes loop, process ~p died with reason ~p", [From, Reason]), + {stop, {From, Reason}, State}; + +handle_info(Msg, #state{init_args = InitArgs} = State) -> + case Msg of + changes_timeout -> + [_, #http_db{url = Url} | _] = InitArgs, + ?LOG_ERROR("changes loop timeout, no data received from ~s", + [couch_util:url_strip_password(Url)]); + _ -> + ?LOG_ERROR("changes loop received unexpected message ~p", [Msg]) + end, + {stop, Msg, State}. terminate(_Reason, State) -> #state{ @@ -154,8 +229,7 @@ terminate(_Reason, State) -> conn = Conn } = State, if is_pid(ChangesPid) -> exit(ChangesPid, stop); true -> ok end, - if is_pid(Conn) -> catch ibrowse:stop_worker_process(Conn); true -> ok end, - ok. + stop_link_worker(Conn). code_change(_OldVsn, State, _Extra) -> {ok, State}. @@ -196,12 +270,17 @@ handle_next_changes(_From, State) -> handle_headers(200, _, State) -> maybe_stream_next(State), {noreply, State}; -handle_headers(301, Hdrs, State) -> - catch ibrowse:stop_worker_process(State#state.conn), - Url = mochiweb_headers:get_value("Location", mochiweb_headers:make(Hdrs)), - %% TODO use couch_httpc:request instead of start_http_request - {Pid, ReqId} = start_http_request(Url), - {noreply, State#state{conn=Pid, reqid=ReqId}}; +handle_headers(Code, Hdrs, #state{init_args = InitArgs} = State) + when Code =:= 301 ; Code =:= 302 -> + stop_link_worker(State#state.conn), + [Parent, #http_db{url = Url1} = Source, Since, PostProps] = InitArgs, + Url = couch_rep_httpc:redirect_url(Hdrs, Url1), + Source2 = couch_rep_httpc:redirected_request(Source, Url), + Pid2 = couch_rep_httpc:spawn_link_worker_process(Source2), + Source3 = Source2#http_db{conn = Pid2}, + {ibrowse_req_id, ReqId} = couch_rep_httpc:request(Source3), + InitArgs2 = [Parent, Source3, Since, PostProps], + {noreply, State#state{conn=Pid2, reqid=ReqId, init_args=InitArgs2}}; handle_headers(Code, Hdrs, State) -> ?LOG_ERROR("replicator changes feed failed with code ~s and Headers ~n~p", [Code,Hdrs]), @@ -212,12 +291,10 @@ handle_messages([], State) -> {noreply, State}; handle_messages([<<"{\"results\":[">>|Rest], State) -> handle_messages(Rest, State); -handle_messages([<<"]">>, <<"\"last_seq\":", LastSeqStr/binary>>], State) -> - LastSeq = list_to_integer(?b2l(hd(re:split(LastSeqStr, "}")))), - handle_feed_completion(State#state{last_seq = LastSeq}); -handle_messages([<<"{\"last_seq\":", LastSeqStr/binary>>], State) -> - LastSeq = list_to_integer(?b2l(hd(re:split(LastSeqStr, "}")))), - handle_feed_completion(State#state{last_seq = LastSeq}); +handle_messages([<<"]">>, <<"\"last_seq\":", _/binary>>], State) -> + handle_feed_completion(State); +handle_messages([<<"{\"last_seq\":", _/binary>>], State) -> + handle_feed_completion(State); handle_messages([Chunk|Rest], State) -> #state{ count = Count, @@ -230,7 +307,7 @@ handle_messages([Chunk|Rest], State) -> #state{reply_to=nil} -> State#state{ count = Count+1, - last_seq = proplists:get_value(<<"seq">>, Props), + last_seq = couch_util:get_value(<<"seq">>, Props), partial_chunk = <<>>, rows=queue:in(Row,Rows) }; @@ -277,16 +354,16 @@ by_seq_loop(Server, Source, StartSeq) -> qs = [{limit, 1000}, {startkey, StartSeq}] }, {Results} = couch_rep_httpc:request(Req), - Rows = proplists:get_value(<<"rows">>, Results), + Rows = couch_util:get_value(<<"rows">>, Results), if Rows =:= [] -> exit(normal); true -> ok end, EndSeq = lists:foldl(fun({RowInfoList}, _) -> - Id = proplists:get_value(<<"id">>, RowInfoList), - Seq = proplists:get_value(<<"key">>, RowInfoList), - {RowProps} = proplists:get_value(<<"value">>, RowInfoList), + Id = couch_util:get_value(<<"id">>, RowInfoList), + Seq = couch_util:get_value(<<"key">>, RowInfoList), + {RowProps} = couch_util:get_value(<<"value">>, RowInfoList), RawRevs = [ - proplists:get_value(<<"rev">>, RowProps), - proplists:get_value(<<"conflicts">>, RowProps, []), - proplists:get_value(<<"deleted_conflicts">>, RowProps, []) + couch_util:get_value(<<"rev">>, RowProps), + couch_util:get_value(<<"conflicts">>, RowProps, []), + couch_util:get_value(<<"deleted_conflicts">>, RowProps, []) ], ParsedRevs = couch_doc:parse_revs(lists:flatten(RawRevs)), Change = {[ @@ -302,73 +379,21 @@ by_seq_loop(Server, Source, StartSeq) -> decode_row(<<",", Rest/binary>>) -> decode_row(Rest); decode_row(Row) -> - {Props} = ?JSON_DECODE(Row), - % [Seq, Id, {<<"changes">>,C}] - Seq = proplists:get_value(<<"seq">>, Props), - Id = proplists:get_value(<<"id">>, Props), - C = proplists:get_value(<<"changes">>, Props), - C2 = [{[{<<"rev">>,couch_doc:parse_rev(R)}]} || {[{<<"rev">>,R}]} <- C], - {[{<<"seq">>, Seq}, {<<"id">>,Id}, {<<"changes">>,C2}]}. - -flush_updated_messages() -> - receive updated -> flush_updated_messages() - after 0 -> ok - end. - -local_update_notification(Self, DbName, {updated, DbName}) -> - Self ! updated; -local_update_notification(_, _, _) -> - ok. + ?JSON_DECODE(Row). maybe_stream_next(#state{reqid=nil}) -> ok; maybe_stream_next(#state{complete=false, count=N} = S) when N < ?BUFFER_SIZE -> + timer:cancel(get(timeout)), + {ok, Timeout} = timer:send_after(31000, changes_timeout), + put(timeout, Timeout), ibrowse:stream_next(S#state.reqid); maybe_stream_next(_) -> - ok. - -send_local_changes_forever(Server, Db, Since) -> - #db{name = DbName, user_ctx = UserCtx} = Db, - {ok, NewSeq} = send_local_changes_once(Server, Db, Since), - couch_db:close(Db), - ok = wait_db_updated(), - {ok, NewDb} = couch_db:open(DbName, [{user_ctx, UserCtx}]), - send_local_changes_forever(Server, NewDb, NewSeq). - -send_local_changes_once(Server, Db, Since) -> - FilterFun = - fun(#doc_info{revs=[#rev_info{rev=Rev}|_]}) -> - {[{<<"rev">>, Rev}]} - end, + timer:cancel(get(timeout)). - ChangesFun = - fun([#doc_info{id=Id, high_seq=Seq}|_]=DocInfos, _) -> - Results0 = [FilterFun(DocInfo) || DocInfo <- DocInfos], - Results = [Result || Result <- Results0, Result /= null], - if Results /= [] -> - Change = {[{<<"seq">>,Seq}, {<<"id">>,Id}, {<<"changes">>,Results}]}, - gen_server:call(Server, {add_change, Change}, infinity); - true -> - ok - end, - {ok, Seq} - end, - - couch_db:changes_since(Db, all_docs, Since, ChangesFun, Since). - -start_http_request(RawUrl) -> - Url = ibrowse_lib:parse_url(RawUrl), - {ok, Pid} = ibrowse:spawn_link_worker_process(Url#url.host, Url#url.port), - Opts = [ - {stream_to, {self(), once}}, - {inactivity_timeout, 31000}, - {response_format, binary} - ], - {ibrowse_req_id, Id} = - ibrowse:send_req_direct(Pid, RawUrl, [], get, [], Opts, infinity), - {Pid, Id}. - -wait_db_updated() -> - receive updated -> - flush_updated_messages() - end. +stop_link_worker(Conn) when is_pid(Conn) -> + unlink(Conn), + receive {'EXIT', Conn, _} -> ok after 0 -> ok end, + catch ibrowse:stop_worker_process(Conn); +stop_link_worker(_) -> + ok. diff --git a/src/couchdb/couch_rep_httpc.erl b/src/couchdb/couch_rep_httpc.erl index 4944f554..ee46a15e 100644 --- a/src/couchdb/couch_rep_httpc.erl +++ b/src/couchdb/couch_rep_httpc.erl @@ -15,7 +15,8 @@ -include("../ibrowse/ibrowse.hrl"). -export([db_exists/1, db_exists/2, full_url/1, request/1, redirected_request/2, - spawn_worker_process/1, spawn_link_worker_process/1]). + redirect_url/2, spawn_worker_process/1, spawn_link_worker_process/1]). +-export([ssl_options/1]). request(#http_db{} = Req) -> do_request(Req). @@ -34,7 +35,7 @@ do_request(Req) -> qs = QS } = Req, Url = full_url(Req), - Headers = case proplists:get_value(<<"oauth">>, Auth) of + Headers = case couch_util:get_value(<<"oauth">>, Auth) of undefined -> Headers0; {OAuthProps} -> @@ -46,7 +47,7 @@ do_request(Req) -> nil -> []; _Else -> - iolist_to_binary(?JSON_ENCODE(B)) + iolist_to_binary(?JSON_ENCODE(B)) end, Resp = case Conn of nil -> @@ -72,10 +73,11 @@ db_exists(Req, CanonicalUrl, CreateDB) -> #http_db{ auth = Auth, headers = Headers0, + options = Options, url = Url } = Req, HeadersFun = fun(Method) -> - case proplists:get_value(<<"oauth">>, Auth) of + case couch_util:get_value(<<"oauth">>, Auth) of undefined -> Headers0; {OAuthProps} -> @@ -84,25 +86,40 @@ db_exists(Req, CanonicalUrl, CreateDB) -> end, case CreateDB of true -> - catch ibrowse:send_req(Url, HeadersFun(put), put); + Headers = [{"Content-Length", 0} | HeadersFun(put)], + catch ibrowse:send_req(Url, Headers, put, [], Options); _Else -> ok end, - case catch ibrowse:send_req(Url, HeadersFun(head), head) of + case catch ibrowse:send_req(Url, HeadersFun(head), head, [], Options) of {ok, "200", _, _} -> Req#http_db{url = CanonicalUrl}; {ok, "301", RespHeaders, _} -> - MochiHeaders = mochiweb_headers:make(RespHeaders), - RedirectUrl = mochiweb_headers:get_value("Location", MochiHeaders), + RedirectUrl = redirect_url(RespHeaders, Req#http_db.url), db_exists(Req#http_db{url = RedirectUrl}, RedirectUrl); {ok, "302", RespHeaders, _} -> - MochiHeaders = mochiweb_headers:make(RespHeaders), - RedirectUrl = mochiweb_headers:get_value("Location", MochiHeaders), + RedirectUrl = redirect_url(RespHeaders, Req#http_db.url), db_exists(Req#http_db{url = RedirectUrl}, CanonicalUrl); + {ok, "401", _, _} -> + throw({unauthorized, ?l2b(Url)}); Error -> ?LOG_DEBUG("DB at ~s could not be found because ~p", [Url, Error]), throw({db_not_found, ?l2b(Url)}) end. +redirect_url(RespHeaders, OrigUrl) -> + MochiHeaders = mochiweb_headers:make(RespHeaders), + RedUrl = mochiweb_headers:get_value("Location", MochiHeaders), + {url, _, Base, Port, _, _, Path, Proto} = ibrowse_lib:parse_url(RedUrl), + {url, _, _, _, User, Passwd, _, _} = ibrowse_lib:parse_url(OrigUrl), + Creds = case is_list(User) andalso is_list(Passwd) of + true -> + User ++ ":" ++ Passwd ++ "@"; + false -> + [] + end, + atom_to_list(Proto) ++ "://" ++ Creds ++ Base ++ ":" ++ + integer_to_list(Port) ++ Path. + full_url(#http_db{url=Url} = Req) when is_binary(Url) -> full_url(Req#http_db{url = ?b2l(Url)}); @@ -115,7 +132,7 @@ full_url(Req) -> resource = Resource, qs = QS } = Req, - QStr = lists:map(fun({K,V}) -> io_lib:format("~s=~s", + QStr = lists:map(fun({K,V}) -> io_lib:format("~s=~s", [couch_util:to_list(K), couch_util:to_list(V)]) end, QS), lists:flatten([Url, Resource, "?", string:join(QStr, "&")]). @@ -124,8 +141,7 @@ process_response({ok, Status, Headers, Body}, Req) -> if Code =:= 200; Code =:= 201 -> ?JSON_DECODE(maybe_decompress(Headers, Body)); Code =:= 301; Code =:= 302 -> - MochiHeaders = mochiweb_headers:make(Headers), - RedirectUrl = mochiweb_headers:get_value("Location", MochiHeaders), + RedirectUrl = redirect_url(Headers, Req#http_db.url), do_request(redirected_request(Req, RedirectUrl)); Code =:= 409 -> throw(conflict); @@ -156,7 +172,7 @@ process_response({error, Reason}, Req) -> pause = Pause } = Req, ShortReason = case Reason of - connection_closed -> + sel_conn_closed -> connection_closed; {'EXIT', {noproc, _}} -> noproc; @@ -178,7 +194,7 @@ process_response({error, Reason}, Req) -> redirected_request(Req, RedirectUrl) -> {Base, QStr, _} = mochiweb_util:urlsplit_path(RedirectUrl), QS = mochiweb_util:parse_qs(QStr), - Hdrs = case proplists:get_value(<<"oauth">>, Req#http_db.auth) of + Hdrs = case couch_util:get_value(<<"oauth">>, Req#http_db.auth) of undefined -> Req#http_db.headers; _Else -> @@ -192,8 +208,7 @@ spawn_worker_process(Req) -> Pid. spawn_link_worker_process(Req) -> - Url = ibrowse_lib:parse_url(Req#http_db.url), - {ok, Pid} = ibrowse_http_client:start_link(Url), + {ok, Pid} = ibrowse:spawn_link_worker_process(Req#http_db.url), Pid. maybe_decompress(Headers, Body) -> @@ -209,11 +224,11 @@ oauth_header(Url, QS, Action, Props) -> % erlang-oauth doesn't like iolists QSL = [{couch_util:to_list(K), ?b2l(?l2b(couch_util:to_list(V)))} || {K,V} <- QS], - ConsumerKey = ?b2l(proplists:get_value(<<"consumer_key">>, Props)), - Token = ?b2l(proplists:get_value(<<"token">>, Props)), - TokenSecret = ?b2l(proplists:get_value(<<"token_secret">>, Props)), - ConsumerSecret = ?b2l(proplists:get_value(<<"consumer_secret">>, Props)), - SignatureMethodStr = ?b2l(proplists:get_value(<<"signature_method">>, Props, <<"HMAC-SHA1">>)), + ConsumerKey = ?b2l(couch_util:get_value(<<"consumer_key">>, Props)), + Token = ?b2l(couch_util:get_value(<<"token">>, Props)), + TokenSecret = ?b2l(couch_util:get_value(<<"token_secret">>, Props)), + ConsumerSecret = ?b2l(couch_util:get_value(<<"consumer_secret">>, Props)), + SignatureMethodStr = ?b2l(couch_util:get_value(<<"signature_method">>, Props, <<"HMAC-SHA1">>)), SignatureMethodAtom = case SignatureMethodStr of "PLAINTEXT" -> plaintext; @@ -232,3 +247,35 @@ oauth_header(Url, QS, Action, Props) -> Params = oauth:signed_params(Method, Url, QSL, Consumer, Token, TokenSecret) -- QSL, {"Authorization", "OAuth " ++ oauth_uri:params_to_header_string(Params)}. + +ssl_options(#http_db{url = Url}) -> + case ibrowse_lib:parse_url(Url) of + #url{protocol = https} -> + Depth = list_to_integer( + couch_config:get("replicator", "ssl_certificate_max_depth", "3") + ), + SslOpts = [{depth, Depth} | + case couch_config:get("replicator", "verify_ssl_certificates") of + "true" -> + ssl_verify_options(true); + _ -> + ssl_verify_options(false) + end], + [{is_ssl, true}, {ssl_options, SslOpts}]; + #url{protocol = http} -> + [] + end. + +ssl_verify_options(Value) -> + ssl_verify_options(Value, erlang:system_info(otp_release)). + +ssl_verify_options(true, OTPVersion) when OTPVersion >= "R14" -> + CAFile = couch_config:get("replicator", "ssl_trusted_certificates_file"), + [{verify, verify_peer}, {cacertfile, CAFile}]; +ssl_verify_options(false, OTPVersion) when OTPVersion >= "R14" -> + [{verify, verify_none}]; +ssl_verify_options(true, _OTPVersion) -> + CAFile = couch_config:get("replicator", "ssl_trusted_certificates_file"), + [{verify, 2}, {cacertfile, CAFile}]; +ssl_verify_options(false, _OTPVersion) -> + [{verify, 0}]. diff --git a/src/couchdb/couch_rep_missing_revs.erl b/src/couchdb/couch_rep_missing_revs.erl index 5790dd71..9809ca5e 100644 --- a/src/couchdb/couch_rep_missing_revs.erl +++ b/src/couchdb/couch_rep_missing_revs.erl @@ -12,7 +12,7 @@ -module(couch_rep_missing_revs). -behaviour(gen_server). --export([init/1, handle_call/3, handle_cast/2, handle_info/2, terminate/2, +-export([init/1, handle_call/3, handle_cast/2, handle_info/2, terminate/2, code_change/3]). -export([start_link/4, next/1, stop/1]). @@ -24,7 +24,6 @@ -record (state, { changes_loop, changes_from = nil, - target, parent, complete = false, count = 0, @@ -44,11 +43,11 @@ next(Server) -> stop(Server) -> gen_server:call(Server, stop). -init([Parent, Target, ChangesFeed, _PostProps]) -> +init([Parent, _Target, ChangesFeed, _PostProps]) -> process_flag(trap_exit, true), Self = self(), - Pid = spawn_link(fun() -> changes_loop(Self, ChangesFeed, Target) end), - {ok, #state{changes_loop=Pid, target=Target, parent=Parent}}. + Pid = spawn_link(fun() -> changes_loop(Self, ChangesFeed, Parent) end), + {ok, #state{changes_loop=Pid, parent=Parent}}. handle_call({add_missing_revs, {HighSeq, Revs}}, From, State) -> State#state.parent ! {update_stats, missing_revs, length(Revs)}, @@ -133,48 +132,57 @@ handle_changes_loop_exit(normal, State) -> handle_changes_loop_exit(Reason, State) -> {stop, Reason, State#state{changes_loop=nil}}. -changes_loop(OurServer, SourceChangesServer, Target) -> +changes_loop(OurServer, SourceChangesServer, Parent) -> case couch_rep_changes_feed:next(SourceChangesServer) of complete -> exit(normal); Changes -> + {ok, Target} = gen_server:call(Parent, get_target_db, infinity), MissingRevs = get_missing_revs(Target, Changes), gen_server:call(OurServer, {add_missing_revs, MissingRevs}, infinity) end, - changes_loop(OurServer, SourceChangesServer, Target). + changes_loop(OurServer, SourceChangesServer, Parent). get_missing_revs(#http_db{}=Target, Changes) -> - Transform = fun({[{<<"seq">>,_}, {<<"id">>,Id}, {<<"changes">>,C}]}) -> - {Id, [couch_doc:rev_to_str(R) || {[{<<"rev">>, R}]} <- C]} end, + Transform = fun({Props}) -> + C = couch_util:get_value(<<"changes">>, Props), + Id = couch_util:get_value(<<"id">>, Props), + {Id, [R || {[{<<"rev">>, R}]} <- C]} + end, IdRevsList = [Transform(Change) || Change <- Changes], SeqDict = changes_dictionary(Changes), - {[{<<"seq">>, HighSeq}, _, _]} = lists:last(Changes), + {LastProps} = lists:last(Changes), + HighSeq = couch_util:get_value(<<"seq">>, LastProps), Request = Target#http_db{ resource = "_missing_revs", method = post, body = {IdRevsList} }, {Resp} = couch_rep_httpc:request(Request), - case proplists:get_value(<<"missing_revs">>, Resp) of + case couch_util:get_value(<<"missing_revs">>, Resp) of {MissingRevs} -> X = [{Id, dict:fetch(Id, SeqDict), couch_doc:parse_revs(RevStrs)} || {Id,RevStrs} <- MissingRevs], {HighSeq, X}; _ -> - exit({target_error, proplists:get_value(<<"error">>, Resp)}) + exit({target_error, couch_util:get_value(<<"error">>, Resp)}) end; get_missing_revs(Target, Changes) -> - Transform = fun({[{<<"seq">>,_}, {<<"id">>,Id}, {<<"changes">>,C}]}) -> - {Id, [R || {[{<<"rev">>, R}]} <- C]} end, + Transform = fun({Props}) -> + C = couch_util:get_value(<<"changes">>, Props), + Id = couch_util:get_value(<<"id">>, Props), + {Id, [couch_doc:parse_rev(R) || {[{<<"rev">>, R}]} <- C]} + end, IdRevsList = [Transform(Change) || Change <- Changes], SeqDict = changes_dictionary(Changes), - {[{<<"seq">>, HighSeq}, _, _]} = lists:last(Changes), + {LastProps} = lists:last(Changes), + HighSeq = couch_util:get_value(<<"seq">>, LastProps), {ok, Results} = couch_db:get_missing_revs(Target, IdRevsList), {HighSeq, [{Id, dict:fetch(Id, SeqDict), Revs} || {Id, Revs, _} <- Results]}. changes_dictionary(ChangeList) -> - KVs = [{proplists:get_value(<<"id">>,C), proplists:get_value(<<"seq">>,C)} + KVs = [{couch_util:get_value(<<"id">>,C), couch_util:get_value(<<"seq">>,C)} || {C} <- ChangeList], dict:from_list(KVs). diff --git a/src/couchdb/couch_rep_reader.erl b/src/couchdb/couch_rep_reader.erl index a66454c8..0930599c 100644 --- a/src/couchdb/couch_rep_reader.erl +++ b/src/couchdb/couch_rep_reader.erl @@ -12,7 +12,7 @@ -module(couch_rep_reader). -behaviour(gen_server). --export([init/1, handle_call/3, handle_cast/2, handle_info/2, terminate/2, +-export([init/1, handle_call/3, handle_cast/2, handle_info/2, terminate/2, code_change/3]). -export([start_link/4, next/1]). @@ -43,13 +43,15 @@ opened_seqs = [] }). -start_link(Parent, Source, MissingRevs, PostProps) -> - gen_server:start_link(?MODULE, [Parent, Source, MissingRevs, PostProps], []). +start_link(Parent, Source, MissingRevs_or_DocIds, PostProps) -> + gen_server:start_link( + ?MODULE, [Parent, Source, MissingRevs_or_DocIds, PostProps], [] + ). next(Pid) -> gen_server:call(Pid, next_docs, infinity). -init([Parent, Source, MissingRevs, _PostProps]) -> +init([Parent, Source, MissingRevs_or_DocIds, _PostProps]) -> process_flag(trap_exit, true), if is_record(Source, http_db) -> #url{host=Host, port=Port} = ibrowse_lib:parse_url(Source#http_db.url), @@ -57,7 +59,15 @@ init([Parent, Source, MissingRevs, _PostProps]) -> ibrowse:set_max_pipeline_size(Host, Port, ?MAX_PIPELINE_SIZE); true -> ok end, Self = self(), - ReaderLoop = spawn_link(fun() -> reader_loop(Self, Source, MissingRevs) end), + ReaderLoop = spawn_link( + fun() -> reader_loop(Self, Parent, Source, MissingRevs_or_DocIds) end + ), + MissingRevs = case MissingRevs_or_DocIds of + Pid when is_pid(Pid) -> + Pid; + _ListDocIds -> + nil + end, State = #state{ parent = Parent, source = Source, @@ -98,6 +108,8 @@ code_change(_OldVsn, State, _Extra) -> %internal funs +handle_add_docs(_Seq, [], _From, State) -> + {reply, ok, State}; handle_add_docs(Seq, DocsToAdd, From, #state{reply_to=nil} = State) -> State1 = update_sequence_lists(Seq, State), NewState = State1#state{ @@ -141,9 +153,13 @@ handle_open_remote_doc(Id, Seq, Revs, _, #state{source=#http_db{}} = State) -> {_, _Ref} = spawn_document_request(Source, Id, Seq, Revs), {reply, ok, State#state{monitor_count = Count+1}}. -handle_monitor_down(normal, #state{pending_doc_request=nil, +handle_monitor_down(normal, #state{pending_doc_request=nil, reply_to=nil, monitor_count=1, complete=waiting_on_monitors} = State) -> {noreply, State#state{complete=true, monitor_count=0}}; +handle_monitor_down(normal, #state{pending_doc_request=nil, reply_to=From, + monitor_count=1, complete=waiting_on_monitors} = State) -> + gen_server:reply(From, {complete, calculate_new_high_seq(State)}), + {stop, normal, State#state{complete=true, monitor_count=0}}; handle_monitor_down(normal, #state{pending_doc_request=nil} = State) -> #state{monitor_count = Count} = State, {noreply, State#state{monitor_count = Count-1}}; @@ -167,6 +183,8 @@ handle_reader_loop_complete(#state{monitor_count=0} = State) -> handle_reader_loop_complete(State) -> {noreply, State#state{complete = waiting_on_monitors}}. +calculate_new_high_seq(#state{missing_revs=nil}) -> + nil; calculate_new_high_seq(#state{requested_seqs=[], opened_seqs=[Open|_]}) -> Open; calculate_new_high_seq(#state{requested_seqs=[Req|_], opened_seqs=[Open|_]}) @@ -191,6 +209,8 @@ split_revlist(Rev, {[CurrentAcc|Rest], BaseLength, Length}) -> % opened seqs greater than the smallest outstanding request. I believe its the % minimal set of info needed to correctly calculate which seqs have been % replicated (because remote docs can be opened out-of-order) -- APK +update_sequence_lists(_Seq, #state{missing_revs=nil} = State) -> + State; update_sequence_lists(Seq, State) -> Requested = lists:delete(Seq, State#state.requested_seqs), AllOpened = lists:merge([Seq], State#state.opened_seqs), @@ -210,10 +230,10 @@ update_sequence_lists(Seq, State) -> opened_seqs = Opened }. -open_doc_revs(#http_db{} = DbS, DocId, Revs) -> +open_doc_revs(#http_db{url = Url} = DbS, DocId, Revs) -> %% all this logic just splits up revision lists that are too long for %% MochiWeb into multiple requests - BaseQS = [{revs,true}, {latest,true}], + BaseQS = [{revs,true}, {latest,true}, {att_encoding_info,true}], BaseReq = DbS#http_db{resource=url_encode(DocId), qs=BaseQS}, BaseLength = length(couch_rep_httpc:full_url(BaseReq)) + 11, % &open_revs= @@ -226,15 +246,61 @@ open_doc_revs(#http_db{} = DbS, DocId, Revs) -> JsonResults = lists:flatten([couch_rep_httpc:request(R) || R <- Requests]), Transform = - fun({[{<<"missing">>, Rev}]}) -> - {{not_found, missing}, couch_doc:parse_rev(Rev)}; - ({[{<<"ok">>, Json}]}) -> + fun({[{<<"ok">>, Json}]}, Acc) -> #doc{id=Id, revs=Rev, atts=Atts} = Doc = couch_doc:from_json_obj(Json), - Doc#doc{atts=[couch_rep_att:convert_stub(A, {DbS,Id,Rev}) || A <- Atts]} + Doc1 = Doc#doc{ + atts=[couch_rep_att:convert_stub(A, {DbS,Id,Rev}) || A <- Atts] + }, + [Doc1 | Acc]; + ({ErrorProps}, Acc) -> + Err = couch_util:get_value(<<"error">>, ErrorProps, + ?JSON_ENCODE({ErrorProps})), + ?LOG_ERROR("Replicator: error accessing doc ~s at ~s, reason: ~s", + [DocId, couch_util:url_strip_password(Url), Err]), + Acc end, - [Transform(Result) || Result <- JsonResults]. + lists:reverse(lists:foldl(Transform, [], JsonResults)). + +open_doc(#http_db{url = Url} = DbS, DocId) -> + % get latest rev of the doc + Req = DbS#http_db{ + resource=url_encode(DocId), + qs=[{att_encoding_info, true}] + }, + {Props} = Json = couch_rep_httpc:request(Req), + case couch_util:get_value(<<"_id">>, Props) of + Id when is_binary(Id) -> + #doc{id=Id, revs=Rev, atts=Atts} = Doc = couch_doc:from_json_obj(Json), + [Doc#doc{ + atts=[couch_rep_att:convert_stub(A, {DbS,Id,Rev}) || A <- Atts] + }]; + undefined -> + Err = couch_util:get_value(<<"error">>, Props, ?JSON_ENCODE(Json)), + ?LOG_ERROR("Replicator: error accessing doc ~s at ~s, reason: ~s", + [DocId, couch_util:url_strip_password(Url), Err]), + [] + end. -reader_loop(ReaderServer, Source, MissingRevsServer) -> +reader_loop(ReaderServer, Parent, Source1, DocIds) when is_list(DocIds) -> + case Source1 of + #http_db{} -> + [gen_server:call(ReaderServer, {open_remote_doc, Id, nil, nil}, + infinity) || Id <- DocIds]; + _LocalDb -> + {ok, Source} = gen_server:call(Parent, get_source_db, infinity), + Docs = lists:foldr(fun(Id, Acc) -> + case couch_db:open_doc(Source, Id) of + {ok, Doc} -> + [Doc | Acc]; + _ -> + Acc + end + end, [], DocIds), + gen_server:call(ReaderServer, {add_docs, nil, Docs}, infinity) + end, + exit(complete); + +reader_loop(ReaderServer, Parent, Source, MissingRevsServer) -> case couch_rep_missing_revs:next(MissingRevsServer) of complete -> exit(complete); @@ -247,26 +313,29 @@ reader_loop(ReaderServer, Source, MissingRevsServer) -> #http_db{} -> [gen_server:call(ReaderServer, {open_remote_doc, Id, Seq, Revs}, infinity) || {Id,Seq,Revs} <- SortedIdsRevs], - reader_loop(ReaderServer, Source, MissingRevsServer); + reader_loop(ReaderServer, Parent, Source, MissingRevsServer); _Local -> - Source2 = maybe_reopen_db(Source, HighSeq), + {ok, Source1} = gen_server:call(Parent, get_source_db, infinity), + Source2 = maybe_reopen_db(Source1, HighSeq), lists:foreach(fun({Id,Seq,Revs}) -> {ok, Docs} = couch_db:open_doc_revs(Source2, Id, Revs, [latest]), JustTheDocs = [Doc || {ok, Doc} <- Docs], gen_server:call(ReaderServer, {add_docs, Seq, JustTheDocs}, infinity) end, SortedIdsRevs), - reader_loop(ReaderServer, Source2, MissingRevsServer) + couch_db:close(Source2), + reader_loop(ReaderServer, Parent, Source2, MissingRevsServer) end end. maybe_reopen_db(#db{update_seq=OldSeq} = Db, HighSeq) when HighSeq > OldSeq -> {ok, NewDb} = couch_db:open(Db#db.name, [{user_ctx, Db#db.user_ctx}]), - couch_db:close(Db), NewDb; maybe_reopen_db(Db, _HighSeq) -> Db. +spawn_document_request(Source, Id, nil, nil) -> + spawn_document_request(Source, Id); spawn_document_request(Source, Id, Seq, Revs) -> Server = self(), SpawnFun = fun() -> @@ -274,3 +343,11 @@ spawn_document_request(Source, Id, Seq, Revs) -> gen_server:call(Server, {add_docs, Seq, Results}, infinity) end, spawn_monitor(SpawnFun). + +spawn_document_request(Source, Id) -> + Server = self(), + SpawnFun = fun() -> + Results = open_doc(Source, Id), + gen_server:call(Server, {add_docs, nil, Results}, infinity) + end, + spawn_monitor(SpawnFun). diff --git a/src/couchdb/couch_rep_writer.erl b/src/couchdb/couch_rep_writer.erl index b86028ce..cf98ccfb 100644 --- a/src/couchdb/couch_rep_writer.erl +++ b/src/couchdb/couch_rep_writer.erl @@ -16,16 +16,19 @@ -include("couch_db.hrl"). -start_link(Parent, Target, Reader, _PostProps) -> - {ok, spawn_link(fun() -> writer_loop(Parent, Reader, Target) end)}. +start_link(Parent, _Target, Reader, _PostProps) -> + {ok, spawn_link(fun() -> writer_loop(Parent, Reader) end)}. -writer_loop(Parent, Reader, Target) -> +writer_loop(Parent, Reader) -> case couch_rep_reader:next(Reader) of + {complete, nil} -> + ok; {complete, FinalSeq} -> Parent ! {writer_checkpoint, FinalSeq}, ok; {HighSeq, Docs} -> DocCount = length(Docs), + {ok, Target} = gen_server:call(Parent, get_target_db, infinity), try write_docs(Target, Docs) of {ok, []} -> Parent ! {update_stats, docs_written, DocCount}; @@ -38,34 +41,131 @@ writer_loop(Parent, Reader, Target) -> ?LOG_DEBUG("writer failed to write an attachment ~p", [Err]), exit({attachment_request_failed, Err, Docs}) end, - Parent ! {writer_checkpoint, HighSeq}, + case HighSeq of + nil -> + ok; + _SeqNumber -> + Parent ! {writer_checkpoint, HighSeq} + end, couch_rep_att:cleanup(), couch_util:should_flush(), - writer_loop(Parent, Reader, Target) + writer_loop(Parent, Reader) end. -write_docs(#http_db{headers = Headers} = Db, Docs) -> - JsonDocs = [couch_doc:to_json_obj(Doc, [revs,attachments]) || Doc <- Docs], +write_docs(#http_db{} = Db, Docs) -> + {DocsAtts, DocsNoAtts} = lists:partition( + fun(#doc{atts=[]}) -> false; (_) -> true end, + Docs + ), + ErrorsJson0 = write_bulk_docs(Db, DocsNoAtts), + ErrorsJson = lists:foldl( + fun(Doc, Acc) -> write_multi_part_doc(Db, Doc) ++ Acc end, + ErrorsJson0, + DocsAtts + ), + {ok, ErrorsJson}; +write_docs(Db, Docs) -> + couch_db:update_docs(Db, Docs, [delay_commit], replicated_changes). + +write_bulk_docs(_Db, []) -> + []; +write_bulk_docs(#http_db{headers = Headers} = Db, Docs) -> + JsonDocs = [ + couch_doc:to_json_obj(Doc, [revs, att_gzip_length]) || Doc <- Docs + ], Request = Db#http_db{ resource = "_bulk_docs", method = post, body = {[{new_edits, false}, {docs, JsonDocs}]}, - headers = [{"x-couch-full-commit", "false"} | Headers] + headers = couch_util:proplist_apply_field({"Content-Type", "application/json"}, [{"X-Couch-Full-Commit", "false"} | Headers]) }, ErrorsJson = case couch_rep_httpc:request(Request) of {FailProps} -> - exit({target_error, proplists:get_value(<<"error">>, FailProps)}); + exit({target_error, couch_util:get_value(<<"error">>, FailProps)}); List when is_list(List) -> List end, - ErrorsList = [write_docs_1(V) || V <- ErrorsJson], - {ok, ErrorsList}; -write_docs(Db, Docs) -> - couch_db:update_docs(Db, Docs, [delay_commit], replicated_changes). + [write_docs_1(V) || V <- ErrorsJson]. + +write_multi_part_doc(#http_db{headers=Headers} = Db, #doc{atts=Atts} = Doc) -> + JsonBytes = ?JSON_ENCODE( + couch_doc:to_json_obj( + Doc, + [follows, att_encoding_info, attachments] + ) + ), + Boundary = couch_uuids:random(), + {ContentType, Len} = couch_doc:len_doc_to_multi_part_stream( + Boundary, JsonBytes, Atts, true + ), + StreamerPid = spawn_link( + fun() -> streamer_fun(Boundary, JsonBytes, Atts) end + ), + BodyFun = fun(Acc) -> + DataQueue = case Acc of + nil -> + StreamerPid ! {start, self()}, + receive + {queue, Q} -> + Q + end; + Queue -> + Queue + end, + case couch_work_queue:dequeue(DataQueue) of + closed -> + eof; + {ok, Data} -> + {ok, iolist_to_binary(Data), DataQueue} + end + end, + Request = Db#http_db{ + resource = couch_util:url_encode(Doc#doc.id), + method = put, + qs = [{new_edits, false}], + body = {BodyFun, nil}, + headers = [ + {"x-couch-full-commit", "false"}, + {"Content-Type", ?b2l(ContentType)}, + {"Content-Length", Len} | Headers + ] + }, + Result = case couch_rep_httpc:request(Request) of + {[{<<"error">>, Error}, {<<"reason">>, Reason}]} -> + {Pos, [RevId | _]} = Doc#doc.revs, + ErrId = couch_util:to_existing_atom(Error), + [{Doc#doc.id, couch_doc:rev_to_str({Pos, RevId})}, {ErrId, Reason}]; + _ -> + [] + end, + StreamerPid ! stop, + Result. + +streamer_fun(Boundary, JsonBytes, Atts) -> + receive + stop -> + ok; + {start, From} -> + % better use a brand new queue, to ensure there's no garbage from + % a previous (failed) iteration + {ok, DataQueue} = couch_work_queue:new(1024 * 1024, 1000), + From ! {queue, DataQueue}, + couch_doc:doc_to_multi_part_stream( + Boundary, + JsonBytes, + Atts, + fun(Data) -> + couch_work_queue:queue(DataQueue, Data) + end, + true + ), + couch_work_queue:close(DataQueue), + streamer_fun(Boundary, JsonBytes, Atts) + end. write_docs_1({Props}) -> - Id = proplists:get_value(<<"id">>, Props), - Rev = couch_doc:parse_rev(proplists:get_value(<<"rev">>, Props)), - ErrId = couch_util:to_existing_atom(proplists:get_value(<<"error">>, Props)), - Reason = proplists:get_value(<<"reason">>, Props), + Id = couch_util:get_value(<<"id">>, Props), + Rev = couch_doc:parse_rev(couch_util:get_value(<<"rev">>, Props)), + ErrId = couch_util:to_existing_atom(couch_util:get_value(<<"error">>, Props)), + Reason = couch_util:get_value(<<"reason">>, Props), {{Id, Rev}, {ErrId, Reason}}. diff --git a/src/couchdb/couch_server.erl b/src/couchdb/couch_server.erl index afdf9365..43fd9044 100644 --- a/src/couchdb/couch_server.erl +++ b/src/couchdb/couch_server.erl @@ -51,30 +51,34 @@ sup_start_link() -> gen_server:start_link({local, couch_server}, couch_server, [], []). open(DbName, Options) -> - case gen_server:call(couch_server, {open, DbName, Options}) of - {ok, MainPid} -> - Ctx = proplists:get_value(user_ctx, Options, #user_ctx{}), - couch_db:open_ref_counted(MainPid, Ctx); + case gen_server:call(couch_server, {open, DbName, Options}, infinity) of + {ok, Db} -> + Ctx = couch_util:get_value(user_ctx, Options, #user_ctx{}), + {ok, Db#db{user_ctx=Ctx}}; Error -> Error end. create(DbName, Options) -> - case gen_server:call(couch_server, {create, DbName, Options}) of - {ok, MainPid} -> - Ctx = proplists:get_value(user_ctx, Options, #user_ctx{}), - couch_db:open_ref_counted(MainPid, Ctx); + case gen_server:call(couch_server, {create, DbName, Options}, infinity) of + {ok, Db} -> + Ctx = couch_util:get_value(user_ctx, Options, #user_ctx{}), + {ok, Db#db{user_ctx=Ctx}}; Error -> Error end. delete(DbName, Options) -> - gen_server:call(couch_server, {delete, DbName, Options}). + gen_server:call(couch_server, {delete, DbName, Options}, infinity). check_dbname(#server{dbname_regexp=RegExp}, DbName) -> case re:run(DbName, RegExp, [{capture, none}]) of nomatch -> - {error, illegal_database_name}; + case DbName of + "_users" -> ok; + _Else -> + {error, illegal_database_name} + end; match -> ok end. @@ -104,7 +108,7 @@ hash_admin_passwords(Persist) -> ({User, ClearPassword}) -> Salt = ?b2l(couch_uuids:random()), Hashed = couch_util:to_hex(crypto:sha(ClearPassword ++ Salt)), - couch_config:set("admins", + couch_config:set("admins", User, "-hashed-" ++ Hashed ++ "," ++ Salt, Persist) end, couch_config:get("admins")). @@ -127,6 +131,7 @@ init([]) -> gen_server:call(couch_server, {set_max_dbs_open, list_to_integer(Max)}) end), + ok = couch_file:init_delete_dir(RootDir), hash_admin_passwords(), ok = couch_config:register( fun("admins", _Key, _Value, Persist) -> @@ -137,22 +142,26 @@ init([]) -> ets:new(couch_dbs_by_name, [set, private, named_table]), ets:new(couch_dbs_by_pid, [set, private, named_table]), ets:new(couch_dbs_by_lru, [ordered_set, private, named_table]), + ets:new(couch_sys_dbs, [set, private, named_table]), process_flag(trap_exit, true), {ok, #server{root_dir=RootDir, dbname_regexp=RegExp, max_dbs_open=MaxDbsOpen, start_time=httpd_util:rfc1123_date()}}. -terminate(Reason, _Srv) -> - couch_util:terminate_linked(Reason), +terminate(_Reason, _Srv) -> + [couch_util:shutdown_sync(Pid) || {_, {Pid, _LruTime}} <- + ets:tab2list(couch_dbs_by_name)], ok. all_databases() -> {ok, #server{root_dir=Root}} = gen_server:call(couch_server, get_server), + NormRoot = couch_util:normpath(Root), Filenames = filelib:fold_files(Root, "^[a-z0-9\\_\\$()\\+\\-]*[\\.]couch$", true, fun(Filename, AccIn) -> - case Filename -- Root of + NormFilename = couch_util:normpath(Filename), + case NormFilename -- NormRoot of [$/ | RelativeFilename] -> ok; RelativeFilename -> ok end, @@ -173,27 +182,27 @@ maybe_close_lru_db(#server{dbs_open=NumOpen}=Server) -> end. try_close_lru(StartTime) -> - LruTime = ets:first(couch_dbs_by_lru), + LruTime = get_lru(), if LruTime > StartTime -> % this means we've looped through all our opened dbs and found them % all in use. {error, all_dbs_active}; true -> [{_, DbName}] = ets:lookup(couch_dbs_by_lru, LruTime), - [{_, {MainPid, LruTime}}] = ets:lookup(couch_dbs_by_name, DbName), + [{_, {opened, MainPid, LruTime}}] = ets:lookup(couch_dbs_by_name, DbName), case couch_db:is_idle(MainPid) of true -> - exit(MainPid, kill), - receive {'EXIT', MainPid, _Reason} -> ok end, + couch_util:shutdown_sync(MainPid), true = ets:delete(couch_dbs_by_lru, LruTime), true = ets:delete(couch_dbs_by_name, DbName), true = ets:delete(couch_dbs_by_pid, MainPid), + true = ets:delete(couch_sys_dbs, DbName), ok; false -> % this still has referrers. Go ahead and give it a current lru time % and try the next one in the table. NewLruTime = now(), - true = ets:insert(couch_dbs_by_name, {DbName, {MainPid, NewLruTime}}), + true = ets:insert(couch_dbs_by_name, {DbName, {opened, MainPid, NewLruTime}}), true = ets:insert(couch_dbs_by_pid, {MainPid, DbName}), true = ets:delete(couch_dbs_by_lru, LruTime), true = ets:insert(couch_dbs_by_lru, {NewLruTime, DbName}), @@ -201,98 +210,150 @@ try_close_lru(StartTime) -> end end. +get_lru() -> + get_lru(ets:first(couch_dbs_by_lru)). + +get_lru(LruTime) -> + [{LruTime, DbName}] = ets:lookup(couch_dbs_by_lru, LruTime), + case ets:member(couch_sys_dbs, DbName) of + false -> + LruTime; + true -> + [{_, {opened, MainPid, _}}] = ets:lookup(couch_dbs_by_name, DbName), + case couch_db:is_idle(MainPid) of + true -> + LruTime; + false -> + get_lru(ets:next(couch_dbs_by_lru, LruTime)) + end + end. + +open_async(Server, From, DbName, Filepath, Options) -> + Parent = self(), + Opener = spawn_link(fun() -> + Res = couch_db:start_link(DbName, Filepath, Options), + gen_server:call( + Parent, {open_result, DbName, Res, Options}, infinity + ), + unlink(Parent), + case Res of + {ok, DbReader} -> + unlink(DbReader); + _ -> + ok + end + end), + true = ets:insert(couch_dbs_by_name, {DbName, {opening, Opener, [From]}}), + true = ets:insert(couch_dbs_by_pid, {Opener, DbName}), + DbsOpen = case lists:member(sys_db, Options) of + true -> + true = ets:insert(couch_sys_dbs, {DbName, true}), + Server#server.dbs_open; + false -> + Server#server.dbs_open + 1 + end, + Server#server{dbs_open = DbsOpen}. + handle_call({set_max_dbs_open, Max}, _From, Server) -> {reply, ok, Server#server{max_dbs_open=Max}}; handle_call(get_server, _From, Server) -> {reply, {ok, Server}, Server}; -handle_call({open, DbName, Options}, _From, Server) -> +handle_call({open_result, DbName, {ok, OpenedDbPid}, Options}, _From, Server) -> + link(OpenedDbPid), + [{DbName, {opening,Opener,Froms}}] = ets:lookup(couch_dbs_by_name, DbName), + lists:foreach(fun({FromPid,_}=From) -> + gen_server:reply(From, + catch couch_db:open_ref_counted(OpenedDbPid, FromPid)) + end, Froms), + LruTime = now(), + true = ets:insert(couch_dbs_by_name, + {DbName, {opened, OpenedDbPid, LruTime}}), + true = ets:delete(couch_dbs_by_pid, Opener), + true = ets:insert(couch_dbs_by_pid, {OpenedDbPid, DbName}), + true = ets:insert(couch_dbs_by_lru, {LruTime, DbName}), + case lists:member(create, Options) of + true -> + couch_db_update_notifier:notify({created, DbName}); + false -> + ok + end, + {reply, ok, Server}; +handle_call({open_result, DbName, Error, Options}, _From, Server) -> + [{DbName, {opening,Opener,Froms}}] = ets:lookup(couch_dbs_by_name, DbName), + lists:foreach(fun(From) -> + gen_server:reply(From, Error) + end, Froms), + true = ets:delete(couch_dbs_by_name, DbName), + true = ets:delete(couch_dbs_by_pid, Opener), + DbsOpen = case lists:member(sys_db, Options) of + true -> + true = ets:delete(couch_sys_dbs, DbName), + Server#server.dbs_open; + false -> + Server#server.dbs_open - 1 + end, + {reply, ok, Server#server{dbs_open = DbsOpen}}; +handle_call({open, DbName, Options}, {FromPid,_}=From, Server) -> LruTime = now(), case ets:lookup(couch_dbs_by_name, DbName) of [] -> - DbNameList = binary_to_list(DbName), - case check_dbname(Server, DbNameList) of - ok -> - case maybe_close_lru_db(Server) of - {ok, Server2} -> - Filepath = get_full_filename(Server, DbNameList), - case couch_db:start_link(DbName, Filepath, Options) of - {ok, MainPid} -> - true = ets:insert(couch_dbs_by_name, {DbName, {MainPid, LruTime}}), - true = ets:insert(couch_dbs_by_pid, {MainPid, DbName}), - true = ets:insert(couch_dbs_by_lru, {LruTime, DbName}), - DbsOpen = Server2#server.dbs_open + 1, - {reply, {ok, MainPid}, - Server2#server{dbs_open=DbsOpen}}; - Error -> - {reply, Error, Server2} - end; - CloseError -> - {reply, CloseError, Server} - end; - Error -> - {reply, Error, Server} - end; - [{_, {MainPid, PrevLruTime}}] -> - true = ets:insert(couch_dbs_by_name, {DbName, {MainPid, LruTime}}), + open_db(DbName, Server, Options, From); + [{_, {opening, Opener, Froms}}] -> + true = ets:insert(couch_dbs_by_name, {DbName, {opening, Opener, [From|Froms]}}), + {noreply, Server}; + [{_, {opened, MainPid, PrevLruTime}}] -> + true = ets:insert(couch_dbs_by_name, {DbName, {opened, MainPid, LruTime}}), true = ets:delete(couch_dbs_by_lru, PrevLruTime), true = ets:insert(couch_dbs_by_lru, {LruTime, DbName}), - {reply, {ok, MainPid}, Server} + {reply, couch_db:open_ref_counted(MainPid, FromPid), Server} end; -handle_call({create, DbName, Options}, _From, Server) -> - DbNameList = binary_to_list(DbName), - case check_dbname(Server, DbNameList) of - ok -> - Filepath = get_full_filename(Server, DbNameList), - - case ets:lookup(couch_dbs_by_name, DbName) of - [] -> - case maybe_close_lru_db(Server) of - {ok, Server2} -> - case couch_db:start_link(DbName, Filepath, [create|Options]) of - {ok, MainPid} -> - LruTime = now(), - true = ets:insert(couch_dbs_by_name, - {DbName, {MainPid, LruTime}}), - true = ets:insert(couch_dbs_by_pid, {MainPid, DbName}), - true = ets:insert(couch_dbs_by_lru, {LruTime, DbName}), - DbsOpen = Server2#server.dbs_open + 1, - couch_db_update_notifier:notify({created, DbName}), - {reply, {ok, MainPid}, - Server2#server{dbs_open=DbsOpen}}; - Error -> - {reply, Error, Server2} - end; - CloseError -> - {reply, CloseError, Server} - end; - [_AlreadyRunningDb] -> - {reply, file_exists, Server} - end; - Error -> - {reply, Error, Server} +handle_call({create, DbName, Options}, From, Server) -> + case ets:lookup(couch_dbs_by_name, DbName) of + [] -> + open_db(DbName, Server, [create | Options], From); + [_AlreadyRunningDb] -> + {reply, file_exists, Server} end; handle_call({delete, DbName, _Options}, _From, Server) -> DbNameList = binary_to_list(DbName), case check_dbname(Server, DbNameList) of ok -> FullFilepath = get_full_filename(Server, DbNameList), - Server2 = + UpdateState = case ets:lookup(couch_dbs_by_name, DbName) of - [] -> Server; - [{_, {Pid, LruTime}}] -> - exit(Pid, kill), - receive {'EXIT', Pid, _Reason} -> ok end, + [] -> false; + [{_, {opening, Pid, Froms}}] -> + couch_util:shutdown_sync(Pid), + true = ets:delete(couch_dbs_by_name, DbName), + true = ets:delete(couch_dbs_by_pid, Pid), + [gen_server:send_result(F, not_found) || F <- Froms], + true; + [{_, {opened, Pid, LruTime}}] -> + couch_util:shutdown_sync(Pid), true = ets:delete(couch_dbs_by_name, DbName), true = ets:delete(couch_dbs_by_pid, Pid), true = ets:delete(couch_dbs_by_lru, LruTime), - Server#server{dbs_open=Server#server.dbs_open - 1} + true + end, + Server2 = case UpdateState of + true -> + DbsOpen = case ets:member(couch_sys_dbs, DbName) of + true -> + true = ets:delete(couch_sys_dbs, DbName), + Server#server.dbs_open; + false -> + Server#server.dbs_open - 1 + end, + Server#server{dbs_open = DbsOpen}; + false -> + Server end, %% Delete any leftover .compact files. If we don't do this a subsequent %% request for this DB will try to open the .compact file and use it. - file:delete(FullFilepath ++ ".compact"), + couch_file:delete(Server#server.root_dir, FullFilepath ++ ".compact"), - case file:delete(FullFilepath) of + case couch_file:delete(Server#server.root_dir, FullFilepath) of ok -> couch_db_update_notifier:notify({deleted, DbName}), {reply, ok, Server2}; @@ -310,15 +371,29 @@ handle_cast(Msg, _Server) -> code_change(_OldVsn, State, _Extra) -> {ok, State}. - -handle_info({'EXIT', _Pid, config_change}, _Server) -> - exit(kill); -handle_info({'EXIT', Pid, _Reason}, #server{dbs_open=DbsOpen}=Server) -> - [{Pid, DbName}] = ets:lookup(couch_dbs_by_pid, Pid), - [{DbName, {Pid, LruTime}}] = ets:lookup(couch_dbs_by_name, DbName), - true = ets:delete(couch_dbs_by_pid, Pid), - true = ets:delete(couch_dbs_by_name, DbName), - true = ets:delete(couch_dbs_by_lru, LruTime), - {noreply, Server#server{dbs_open=DbsOpen - 1}}; -handle_info(Info, _Server) -> - exit({unknown_message, Info}). + +handle_info({'EXIT', _Pid, config_change}, Server) -> + {noreply, shutdown, Server}; +handle_info(Error, _Server) -> + ?LOG_ERROR("Unexpected message, restarting couch_server: ~p", [Error]), + exit(kill). + +open_db(DbName, Server, Options, From) -> + DbNameList = binary_to_list(DbName), + case check_dbname(Server, DbNameList) of + ok -> + Filepath = get_full_filename(Server, DbNameList), + case lists:member(sys_db, Options) of + true -> + {noreply, open_async(Server, From, DbName, Filepath, Options)}; + false -> + case maybe_close_lru_db(Server) of + {ok, Server2} -> + {noreply, open_async(Server2, From, DbName, Filepath, Options)}; + CloseError -> + {reply, CloseError, Server} + end + end; + Error -> + {reply, Error, Server} + end. diff --git a/src/couchdb/couch_server_sup.erl b/src/couchdb/couch_server_sup.erl index d89e987d..4f0445da 100644 --- a/src/couchdb/couch_server_sup.erl +++ b/src/couchdb/couch_server_sup.erl @@ -32,12 +32,7 @@ start_link(IniFiles) -> end. restart_core_server() -> - supervisor:terminate_child(couch_primary_services, couch_server), - supervisor:terminate_child(couch_secondary_services, stats_aggregator), - supervisor:terminate_child(couch_secondary_services, stats_collector), - supervisor:restart_child(couch_primary_services, couch_server), - supervisor:restart_child(couch_secondary_services, stats_collector), - supervisor:restart_child(couch_secondary_services, stats_aggregator). + init:restart(). couch_config_start_link_wrapper(IniFiles, FirstConfigPid) -> case is_process_alive(FirstConfigPid) of @@ -127,6 +122,13 @@ start_server(IniFiles) -> Port = mochiweb_socket_server:get(couch_httpd, port), io:format("Apache CouchDB has started. Time to relax.~n"), ?LOG_INFO("Apache CouchDB has started on http://~s:~w/", [Ip, Port]), + + case couch_config:get("couchdb", "uri_file", null) of + null -> ok; + UriFile -> + Line = io_lib:format("http://~s:~w/~n", [Ip, Port]), + file:write_file(UriFile, Line) + end, {ok, Pid}. @@ -174,7 +176,7 @@ start_secondary_services() -> {list_to_atom(Name), {Module, Fun, Args}, permanent, - brutal_kill, + 1000, worker, [Module]} end diff --git a/src/couchdb/couch_stats_collector.erl b/src/couchdb/couch_stats_collector.erl index 59d62a6e..f7b9bb48 100644 --- a/src/couchdb/couch_stats_collector.erl +++ b/src/couchdb/couch_stats_collector.erl @@ -60,7 +60,7 @@ increment(Key) -> Key2 = make_key(Key), case catch ets:update_counter(?HIT_TABLE, Key2, 1) of {'EXIT', {badarg, _}} -> - true = ets:insert(?HIT_TABLE, {Key2, 1}), + catch ets:insert(?HIT_TABLE, {Key2, 1}), ok; _ -> ok @@ -70,16 +70,16 @@ decrement(Key) -> Key2 = make_key(Key), case catch ets:update_counter(?HIT_TABLE, Key2, -1) of {'EXIT', {badarg, _}} -> - true = ets:insert(?HIT_TABLE, {Key2, -1}), + catch ets:insert(?HIT_TABLE, {Key2, -1}), ok; _ -> ok end. record(Key, Value) -> - true = ets:insert(?ABS_TABLE, {make_key(Key), Value}). + catch ets:insert(?ABS_TABLE, {make_key(Key), Value}). clear(Key) -> - true = ets:delete(?ABS_TABLE, make_key(Key)). + catch ets:delete(?ABS_TABLE, make_key(Key)). track_process_count(Stat) -> track_process_count(self(), Stat). diff --git a/src/couchdb/couch_stream.erl b/src/couchdb/couch_stream.erl index 2a873e4c..04c17770 100644 --- a/src/couchdb/couch_stream.erl +++ b/src/couchdb/couch_stream.erl @@ -24,7 +24,7 @@ -define(DEFAULT_STREAM_CHUNK, 16#00100000). % 1 meg chunks when streaming data --export([open/1, close/1, write/2, foldl/4, foldl/5, +-export([open/1, open/3, close/1, write/2, foldl/4, foldl/5, foldl_decode/6, old_foldl/5,old_copy_to_new_stream/4]). -export([copy_to_new_stream/3,old_read_term/2]). -export([init/1, terminate/2, handle_call/3]). @@ -39,14 +39,23 @@ buffer_len = 0, max_buffer = 4096, written_len = 0, - md5 + md5, + % md5 of the content without any transformation applied (e.g. compression) + % needed for the attachment upload integrity check (ticket 558) + identity_md5, + identity_len = 0, + encoding_fun, + end_encoding_fun }). %%% Interface functions %%% open(Fd) -> - gen_server:start_link(couch_stream, Fd, []). + open(Fd, identity, []). + +open(Fd, Encoding, Options) -> + gen_server:start_link(couch_stream, {Fd, Encoding, Options}, []). close(Pid) -> gen_server:call(Pid, close, infinity). @@ -85,19 +94,86 @@ foldl(Fd, [Pos|Rest], Fun, Acc) -> foldl(Fd, PosList, <<>>, Fun, Acc) -> foldl(Fd, PosList, Fun, Acc); foldl(Fd, PosList, Md5, Fun, Acc) -> - foldl(Fd, PosList, Md5, erlang:md5_init(), Fun, Acc). - + foldl(Fd, PosList, Md5, couch_util:md5_init(), Fun, Acc). + +foldl_decode(Fd, PosList, Md5, Enc, Fun, Acc) -> + {DecDataFun, DecEndFun} = case Enc of + gzip -> + ungzip_init(); + identity -> + identity_enc_dec_funs() + end, + Result = foldl_decode( + DecDataFun, Fd, PosList, Md5, couch_util:md5_init(), Fun, Acc + ), + DecEndFun(), + Result. foldl(_Fd, [], Md5, Md5Acc, _Fun, Acc) -> - Md5 = erlang:md5_final(Md5Acc), + Md5 = couch_util:md5_final(Md5Acc), Acc; foldl(Fd, [Pos], Md5, Md5Acc, Fun, Acc) -> {ok, Bin} = couch_file:pread_iolist(Fd, Pos), - Md5 = erlang:md5_final(erlang:md5_update(Md5Acc, Bin)), + Md5 = couch_util:md5_final(couch_util:md5_update(Md5Acc, Bin)), Fun(Bin, Acc); foldl(Fd, [Pos|Rest], Md5, Md5Acc, Fun, Acc) -> {ok, Bin} = couch_file:pread_iolist(Fd, Pos), - foldl(Fd, Rest, Md5, erlang:md5_update(Md5Acc, Bin), Fun, Fun(Bin, Acc)). + foldl(Fd, Rest, Md5, couch_util:md5_update(Md5Acc, Bin), Fun, Fun(Bin, Acc)). + +foldl_decode(_DecFun, _Fd, [], Md5, Md5Acc, _Fun, Acc) -> + Md5 = couch_util:md5_final(Md5Acc), + Acc; +foldl_decode(DecFun, Fd, [Pos], Md5, Md5Acc, Fun, Acc) -> + {ok, EncBin} = couch_file:pread_iolist(Fd, Pos), + Md5 = couch_util:md5_final(couch_util:md5_update(Md5Acc, EncBin)), + Bin = DecFun(EncBin), + Fun(Bin, Acc); +foldl_decode(DecFun, Fd, [Pos|Rest], Md5, Md5Acc, Fun, Acc) -> + {ok, EncBin} = couch_file:pread_iolist(Fd, Pos), + Bin = DecFun(EncBin), + Md5Acc2 = couch_util:md5_update(Md5Acc, EncBin), + foldl_decode(DecFun, Fd, Rest, Md5, Md5Acc2, Fun, Fun(Bin, Acc)). + +gzip_init(Options) -> + case couch_util:get_value(compression_level, Options, 0) of + Lvl when Lvl >= 1 andalso Lvl =< 9 -> + Z = zlib:open(), + % 15 = ?MAX_WBITS (defined in the zlib module) + % the 16 + ?MAX_WBITS formula was obtained by inspecting zlib:gzip/1 + ok = zlib:deflateInit(Z, Lvl, deflated, 16 + 15, 8, default), + { + fun(Data) -> + zlib:deflate(Z, Data) + end, + fun() -> + Last = zlib:deflate(Z, [], finish), + ok = zlib:deflateEnd(Z), + ok = zlib:close(Z), + Last + end + }; + _ -> + identity_enc_dec_funs() + end. + +ungzip_init() -> + Z = zlib:open(), + zlib:inflateInit(Z, 16 + 15), + { + fun(Data) -> + zlib:inflate(Z, Data) + end, + fun() -> + ok = zlib:inflateEnd(Z), + ok = zlib:close(Z) + end + }. + +identity_enc_dec_funs() -> + { + fun(Data) -> Data end, + fun() -> [] end + }. write(_Pid, <<>>) -> ok; @@ -105,8 +181,21 @@ write(Pid, Bin) -> gen_server:call(Pid, {write, Bin}, infinity). -init(Fd) -> - {ok, #stream{fd=Fd, md5=erlang:md5_init()}}. +init({Fd, Encoding, Options}) -> + {EncodingFun, EndEncodingFun} = case Encoding of + identity -> + identity_enc_dec_funs(); + gzip -> + gzip_init(Options) + end, + {ok, #stream{ + fd=Fd, + md5=couch_util:md5_init(), + identity_md5=couch_util:md5_init(), + encoding_fun=EncodingFun, + end_encoding_fun=EndEncodingFun + } + }. terminate(_Reason, _Stream) -> ok. @@ -120,39 +209,65 @@ handle_call({write, Bin}, _From, Stream) -> buffer_len = BufferLen, buffer_list = Buffer, max_buffer = Max, - md5 = Md5} = Stream, + md5 = Md5, + identity_md5 = IdenMd5, + identity_len = IdenLen, + encoding_fun = EncodingFun} = Stream, if BinSize + BufferLen > Max -> WriteBin = lists:reverse(Buffer, [Bin]), - Md5_2 = erlang:md5_update(Md5, WriteBin), - {ok, Pos} = couch_file:append_binary(Fd, WriteBin), + IdenMd5_2 = couch_util:md5_update(IdenMd5, WriteBin), + case EncodingFun(WriteBin) of + [] -> + % case where the encoder did some internal buffering + % (zlib does it for example) + WrittenLen2 = WrittenLen, + Md5_2 = Md5, + Written2 = Written; + WriteBin2 -> + {ok, Pos} = couch_file:append_binary(Fd, WriteBin2), + WrittenLen2 = WrittenLen + iolist_size(WriteBin2), + Md5_2 = couch_util:md5_update(Md5, WriteBin2), + Written2 = [Pos|Written] + end, + {reply, ok, Stream#stream{ - written_len=WrittenLen + BufferLen + BinSize, - written_pointers=[Pos|Written], + written_len=WrittenLen2, + written_pointers=Written2, buffer_list=[], buffer_len=0, - md5=Md5_2}}; + md5=Md5_2, + identity_md5=IdenMd5_2, + identity_len=IdenLen + BinSize}}; true -> {reply, ok, Stream#stream{ buffer_list=[Bin|Buffer], - buffer_len=BufferLen + BinSize}} + buffer_len=BufferLen + BinSize, + identity_len=IdenLen + BinSize}} end; handle_call(close, _From, Stream) -> #stream{ fd = Fd, written_len = WrittenLen, written_pointers = Written, - buffer_len = BufferLen, buffer_list = Buffer, - md5 = Md5} = Stream, - - case Buffer of + md5 = Md5, + identity_md5 = IdenMd5, + identity_len = IdenLen, + encoding_fun = EncodingFun, + end_encoding_fun = EndEncodingFun} = Stream, + + WriteBin = lists:reverse(Buffer), + IdenMd5Final = couch_util:md5_final(couch_util:md5_update(IdenMd5, WriteBin)), + WriteBin2 = EncodingFun(WriteBin) ++ EndEncodingFun(), + Md5Final = couch_util:md5_final(couch_util:md5_update(Md5, WriteBin2)), + Result = case WriteBin2 of [] -> - Result = {lists:reverse(Written), WrittenLen, erlang:md5_final(Md5)}; + {lists:reverse(Written), WrittenLen, IdenLen, Md5Final, IdenMd5Final}; _ -> - WriteBin = lists:reverse(Buffer), - Md5Final = erlang:md5_final(erlang:md5_update(Md5, WriteBin)), - {ok, Pos} = couch_file:append_binary(Fd, WriteBin), - Result = {lists:reverse(Written, [Pos]), WrittenLen + BufferLen, Md5Final} + {ok, Pos} = couch_file:append_binary(Fd, WriteBin2), + StreamInfo = lists:reverse(Written, [Pos]), + StreamLen = WrittenLen + iolist_size(WriteBin2), + {StreamInfo, StreamLen, IdenLen, Md5Final, IdenMd5Final} end, {stop, normal, Result, Stream}. diff --git a/src/couchdb/couch_util.erl b/src/couchdb/couch_util.erl index 6edfb781..7a8ae055 100644 --- a/src/couchdb/couch_util.erl +++ b/src/couchdb/couch_util.erl @@ -12,15 +12,22 @@ -module(couch_util). --export([priv_dir/0, start_driver/1,terminate_linked/1]). +-export([priv_dir/0, start_driver/1, normpath/1]). -export([should_flush/0, should_flush/1, to_existing_atom/1]). -export([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, encodeBase64Url/1, decodeBase64Url/1, - to_hex/1,parse_term/1, dict_find/3]). +-export([encodeBase64Url/1, decodeBase64Url/1]). +-export([to_hex/1, parse_term/1, dict_find/3]). -export([file_read_size/1, get_nested_json_value/2, json_user_ctx/1]). +-export([proplist_apply_field/2, json_apply_field/2]). -export([to_binary/1, to_integer/1, to_list/1, url_encode/1]). -export([json_encode/1, json_decode/1]). +-export([verify/2,simple_call/2,shutdown_sync/1]). +-export([compressible_att_type/1]). +-export([get_value/2, get_value/3]). +-export([md5/1, md5_init/0, md5_update/2, md5_final/1]). +-export([reorder_results/2]). +-export([url_strip_password/1]). -include("couch_db.hrl"). -include_lib("kernel/include/file.hrl"). @@ -48,23 +55,57 @@ start_driver(LibDir) -> exit(erl_ddll:format_error(Error)) end. +% Normalize a pathname by removing .. and . components. +normpath(Path) -> + normparts(filename:split(Path), []). + +normparts([], Acc) -> + filename:join(lists:reverse(Acc)); +normparts([".." | RestParts], [_Drop | RestAcc]) -> + normparts(RestParts, RestAcc); +normparts(["." | RestParts], Acc) -> + normparts(RestParts, Acc); +normparts([Part | RestParts], Acc) -> + normparts(RestParts, [Part | Acc]). + % 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; + 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; + try list_to_existing_atom(?b2l(V)) catch _:_ -> V end; to_existing_atom(V) when is_atom(V) -> V. +shutdown_sync(Pid) when not is_pid(Pid)-> + ok; +shutdown_sync(Pid) -> + MRef = erlang:monitor(process, Pid), + try + catch unlink(Pid), + catch exit(Pid, shutdown), + receive + {'DOWN', MRef, _, _, _} -> + ok + end + after + erlang:demonitor(MRef, [flush]) + end. + -terminate_linked(normal) -> - terminate_linked(shutdown); -terminate_linked(Reason) -> - {links, Links} = process_info(self(), links), - [catch exit(Pid, Reason) || Pid <- Links], - ok. - +simple_call(Pid, Message) -> + MRef = erlang:monitor(process, Pid), + try + Pid ! {self(), Message}, + receive + {Pid, Result} -> + Result; + {'DOWN', MRef, _, _, Reason} -> + exit(Reason) + end + after + erlang:demonitor(MRef, [flush]) + end. to_hex([]) -> []; @@ -83,9 +124,19 @@ parse_term(List) -> {ok, Tokens, _} = erl_scan:string(List ++ "."), erl_parse:parse_term(Tokens). +get_value(Key, List) -> + get_value(Key, List, undefined). + +get_value(Key, List, Default) -> + case lists:keysearch(Key, 1, List) of + {value, {Key,Value}} -> + Value; + false -> + Default + end. get_nested_json_value({Props}, [Key|Keys]) -> - case proplists:get_value(Key, Props, nil) of + case couch_util:get_value(Key, Props, nil) of nil -> throw({not_found, <<"missing json key: ", Key/binary>>}); Value -> get_nested_json_value(Value, Keys) end; @@ -94,6 +145,19 @@ get_nested_json_value(Value, []) -> get_nested_json_value(_NotJSONObj, _) -> throw({not_found, json_mismatch}). +proplist_apply_field(H, L) -> + {R} = json_apply_field(H, {L}), + R. + +json_apply_field(H, {L}) -> + json_apply_field(H, L, []). +json_apply_field({Key, NewValue}, [{Key, _OldVal} | Headers], Acc) -> + json_apply_field({Key, NewValue}, Headers, Acc); +json_apply_field({Key, NewValue}, [{OtherKey, OtherVal} | Headers], Acc) -> + json_apply_field({Key, NewValue}, Headers, [{OtherKey, OtherVal} | Acc]); +json_apply_field({Key, NewValue}, [], Acc) -> + {[{Key, NewValue}|Acc]}. + json_user_ctx(#db{name=DbName, user_ctx=Ctx}) -> {[{<<"db">>, DbName}, {<<"name">>,Ctx#user_ctx.name}, @@ -230,114 +294,18 @@ should_flush(MemThreshHold) -> ProcMem2+BinMem2 > MemThreshHold; true -> false end. +encodeBase64Url(Url) -> + Url1 = iolist_to_binary(re:replace(base64:encode(Url), "=+$", "")), + Url2 = iolist_to_binary(re:replace(Url1, "/", "_", [global])), + iolist_to_binary(re:replace(Url2, "\\+", "-", [global])). -%%% Purpose : Base 64 encoding and decoding. -%%% Copied from ssl_base_64 to avoid using the -%%% erlang ssl library - --define(st(X,A), ((X-A+256) div 256)). - -%% A PEM encoding consists of characters A-Z, a-z, 0-9, +, / and -%% =. Each character encodes a 6 bits value from 0 to 63 (A = 0, / = -%% 63); = is a padding character. -%% - -%% -%% encode64(Bytes|Binary) -> binary -%% -%% Take 3 bytes a time (3 x 8 = 24 bits), and make 4 characters out of -%% them (4 x 6 = 24 bits). -%% -encodeBase64(Bs) when is_list(Bs) -> - encodeBase64(iolist_to_binary(Bs), <<>>); -encodeBase64(Bs) -> - encodeBase64(Bs, <<>>). - -encodeBase64(<<B:3/binary, Bs/binary>>, Acc) -> - <<C1:6, C2:6, C3:6, C4:6>> = B, - encodeBase64(Bs, <<Acc/binary, (enc(C1)), (enc(C2)), (enc(C3)), (enc(C4))>>); -encodeBase64(<<B:2/binary>>, Acc) -> - <<C1:6, C2:6, C3:6, _:6>> = <<B/binary, 0>>, - <<Acc/binary, (enc(C1)), (enc(C2)), (enc(C3)), $=>>; -encodeBase64(<<B:1/binary>>, Acc) -> - <<C1:6, C2:6, _:12>> = <<B/binary, 0, 0>>, - <<Acc/binary, (enc(C1)), (enc(C2)), $=, $=>>; -encodeBase64(<<>>, Acc) -> - Acc. - -encodeBase64Url(Bs) when is_list(Bs) -> - encodeBase64Url(list_to_binary(Bs), <<>>); -encodeBase64Url(Bs) -> - encodeBase64Url(Bs, <<>>). - -encodeBase64Url(<<B:3/binary, Bs/binary>>, Acc) -> - <<C1:6, C2:6, C3:6, C4:6>> = B, - encodeBase64Url(Bs, <<Acc/binary, (encUrl(C1)), (encUrl(C2)), (encUrl(C3)), (encUrl(C4))>>); -encodeBase64Url(<<B:2/binary>>, Acc) -> - <<C1:6, C2:6, C3:6, _:6>> = <<B/binary, 0>>, - <<Acc/binary, (encUrl(C1)), (encUrl(C2)), (encUrl(C3))>>; -encodeBase64Url(<<B:1/binary>>, Acc) -> - <<C1:6, C2:6, _:12>> = <<B/binary, 0, 0>>, - <<Acc/binary, (encUrl(C1)), (encUrl(C2))>>; -encodeBase64Url(<<>>, Acc) -> - Acc. - -%% -%% decodeBase64(BinaryChars) -> Binary -%% -decodeBase64(Cs) when is_list(Cs) -> - decodeBase64(list_to_binary(Cs)); -decodeBase64(Cs) -> - decode1(Cs, <<>>). - -decode1(<<C1, C2, $=, $=>>, Acc) -> - <<B1, _:16>> = <<(dec(C1)):6, (dec(C2)):6, 0:12>>, - <<Acc/binary, B1>>; -decode1(<<C1, C2, C3, $=>>, Acc) -> - <<B1, B2, _:8>> = <<(dec(C1)):6, (dec(C2)):6, (dec(C3)):6, (dec(0)):6>>, - <<Acc/binary, B1, B2>>; -decode1(<<C1, C2, C3, C4, Cs/binary>>, Acc) -> - Bin = <<Acc/binary, (dec(C1)):6, (dec(C2)):6, (dec(C3)):6, (dec(C4)):6>>, - decode1(Cs, Bin); -decode1(<<>>, Acc) -> - Acc. - -decodeBase64Url(Cs) when is_list(Cs) -> - decodeBase64Url(list_to_binary(Cs)); -decodeBase64Url(Cs) -> - decode1Url(Cs, <<>>). - -decode1Url(<<C1, C2>>, Acc) -> - <<B1, _:16>> = <<(decUrl(C1)):6, (decUrl(C2)):6, 0:12>>, - <<Acc/binary, B1>>; -decode1Url(<<C1, C2, C3>>, Acc) -> - <<B1, B2, _:8>> = <<(decUrl(C1)):6, (decUrl(C2)):6, (decUrl(C3)):6, (decUrl(0)):6>>, - <<Acc/binary, B1, B2>>; -decode1Url(<<C1, C2, C3, C4, Cs/binary>>, Acc) -> - Bin = <<Acc/binary, (decUrl(C1)):6, (decUrl(C2)):6, (decUrl(C3)):6, (decUrl(C4)):6>>, - decode1Url(Cs, Bin); -decode1Url(<<>>, Acc) -> - Acc. - -%% enc/1 and dec/1 -%% -%% Mapping: 0-25 -> A-Z, 26-51 -> a-z, 52-61 -> 0-9, 62 -> +, 63 -> / -%% -enc(C) -> - 65 + C + 6*?st(C,26) - 75*?st(C,52) -15*?st(C,62) + 3*?st(C,63). - -dec(C) -> - 62*?st(C,43) + ?st(C,47) + (C-59)*?st(C,48) - 69*?st(C,65) - 6*?st(C,97). - -%% encUrl/1 and decUrl/1 -%% -%% Mapping: 0-25 -> A-Z, 26-51 -> a-z, 52-61 -> 0-9, 62 -> -, 63 -> _ -%% -encUrl(C) -> - 65 + C + 6*?st(C,26) - 75*?st(C,52) -13*?st(C,62) + 49*?st(C,63). - -decUrl(C) -> - 62*?st(C,45) + (C-58)*?st(C,48) - 69*?st(C,65) + 33*?st(C,95) - 39*?st(C,97). +decodeBase64Url(Url64) -> + Url1 = re:replace(iolist_to_binary(Url64), "-", "+", [global]), + Url2 = iolist_to_binary( + re:replace(iolist_to_binary(Url1), "_", "/", [global]) + ), + Padding = ?l2b(lists:duplicate((4 - size(Url2) rem 4) rem 4, $=)), + base64:decode(<<Url2/binary, Padding/binary>>). dict_find(Key, Dict, DefaultValue) -> case dict:find(Key, Dict) of @@ -423,3 +391,71 @@ json_decode(V) -> _Type:_Error -> throw({invalid_json,V}) end. + +verify([X|RestX], [Y|RestY], Result) -> + verify(RestX, RestY, (X bxor Y) bor Result); +verify([], [], Result) -> + Result == 0. + +verify(<<X/binary>>, <<Y/binary>>) -> + verify(?b2l(X), ?b2l(Y)); +verify(X, Y) when is_list(X) and is_list(Y) -> + case length(X) == length(Y) of + true -> + verify(X, Y, 0); + false -> + false + end; +verify(_X, _Y) -> false. + +compressible_att_type(MimeType) when is_binary(MimeType) -> + compressible_att_type(?b2l(MimeType)); +compressible_att_type(MimeType) -> + TypeExpList = re:split( + couch_config:get("attachments", "compressible_types", ""), + ", ?", + [{return, list}] + ), + lists:any( + fun(TypeExp) -> + Regexp = "^\\s*" ++ + re:replace(TypeExp, "\\*", ".*", [{return, list}]) ++ "\\s*$", + case re:run(MimeType, Regexp, [caseless]) of + {match, _} -> + true; + _ -> + false + end + end, + [T || T <- TypeExpList, T /= []] + ). + +-spec md5(Data::(iolist() | binary())) -> Digest::binary(). +md5(Data) -> + try crypto:md5(Data) catch error:_ -> erlang:md5(Data) end. + +-spec md5_init() -> Context::binary(). +md5_init() -> + try crypto:md5_init() catch error:_ -> erlang:md5_init() end. + +-spec md5_update(Context::binary(), Data::(iolist() | binary())) -> + NewContext::binary(). +md5_update(Ctx, D) -> + try crypto:md5_update(Ctx,D) catch error:_ -> erlang:md5_update(Ctx,D) end. + +-spec md5_final(Context::binary()) -> Digest::binary(). +md5_final(Ctx) -> + try crypto:md5_final(Ctx) catch error:_ -> erlang:md5_final(Ctx) end. + +% linear search is faster for small lists, length() is 0.5 ms for 100k list +reorder_results(Keys, SortedResults) when length(Keys) < 100 -> + [couch_util:get_value(Key, SortedResults) || Key <- Keys]; +reorder_results(Keys, SortedResults) -> + KeyDict = dict:from_list(SortedResults), + [dict:fetch(Key, KeyDict) || Key <- Keys]. + +url_strip_password(Url) -> + re:replace(Url, + "http(s)?://([^:]+):[^@]+@(.*)$", + "http\\1://\\2:*****@\\3", + [{return, list}]). diff --git a/src/couchdb/couch_view.erl b/src/couchdb/couch_view.erl index f80ce434..ebd7a2d4 100644 --- a/src/couchdb/couch_view.erl +++ b/src/couchdb/couch_view.erl @@ -29,11 +29,9 @@ start_link() -> gen_server:start_link({local, couch_view}, couch_view, [], []). get_temp_updater(DbName, Language, DesignOptions, MapSrc, RedSrc) -> - % make temp group - % do we need to close this db? - {ok, _Db, Group} = + {ok, Group} = couch_view_group:open_temp_group(DbName, Language, DesignOptions, MapSrc, RedSrc), - case gen_server:call(couch_view, {get_group_server, DbName, Group}) of + case gen_server:call(couch_view, {get_group_server, DbName, Group}, infinity) of {ok, Pid} -> Pid; Error -> @@ -41,11 +39,9 @@ get_temp_updater(DbName, Language, DesignOptions, MapSrc, RedSrc) -> end. get_group_server(DbName, GroupId) -> - % get signature for group case couch_view_group:open_db_group(DbName, GroupId) of - % do we need to close this db? - {ok, _Db, Group} -> - case gen_server:call(couch_view, {get_group_server, DbName, Group}) of + {ok, Group} -> + case gen_server:call(couch_view, {get_group_server, DbName, Group}, infinity) of {ok, Pid} -> Pid; Error -> @@ -80,7 +76,7 @@ cleanup_index_files(Db) -> % make unique list of group sigs Sigs = lists:map(fun(#doc{id = GroupId}) -> {ok, Info} = get_group_info(Db, GroupId), - ?b2l(proplists:get_value(signature, Info)) + ?b2l(couch_util:get_value(signature, Info)) end, [DD||DD <- DesignDocs, DD#doc.deleted == false]), FileList = list_index_files(Db), @@ -90,11 +86,12 @@ cleanup_index_files(Db) -> % filter out the ones in use DeleteFiles = [FilePath - || FilePath <- FileList, - re:run(FilePath, RegExp, [{capture, none}]) =:= nomatch], + || FilePath <- FileList, + re:run(FilePath, RegExp, [{capture, none}]) =:= nomatch], % delete unused files ?LOG_DEBUG("deleting unused view index files: ~p",[DeleteFiles]), - [file:delete(File)||File <- DeleteFiles], + RootDir = couch_config:get("couchdb", "view_index_dir"), + [couch_file:delete(RootDir,File,false)||File <- DeleteFiles], ok. list_index_files(Db) -> @@ -266,11 +263,13 @@ init([]) -> ets:new(group_servers_by_sig, [set, protected, named_table]), ets:new(couch_groups_by_updater, [set, private, named_table]), process_flag(trap_exit, true), + ok = couch_file:init_delete_dir(RootDir), {ok, #server{root_dir=RootDir}}. -terminate(Reason, _Srv) -> - couch_util:terminate_linked(Reason), +terminate(_Reason, _Srv) -> + [couch_util:shutdown_sync(Pid) || {Pid, _} <- + ets:tab2list(couch_groups_by_updater)], ok. @@ -284,6 +283,15 @@ handle_call({get_group_server, DbName, {ok, NewPid} -> add_to_ets(NewPid, DbName, Sig), {reply, {ok, NewPid}, Server}; + {error, invalid_view_seq} -> + do_reset_indexes(DbName, Root), + case (catch couch_view_group:start_link({Root, DbName, Group})) of + {ok, NewPid} -> + add_to_ets(NewPid, DbName, Sig), + {reply, {ok, NewPid}, Server}; + Error -> + {reply, Error, Server} + end; Error -> {reply, Error, Server} end; @@ -292,20 +300,22 @@ handle_call({get_group_server, DbName, end. handle_cast({reset_indexes, DbName}, #server{root_dir=Root}=Server) -> + do_reset_indexes(DbName, Root), + {noreply, Server}. + +do_reset_indexes(DbName, Root) -> % shutdown all the updaters and clear the files, the db got changed Names = ets:lookup(couch_groups_by_db, DbName), lists:foreach( fun({_DbName, Sig}) -> ?LOG_DEBUG("Killing update process for view group ~s. in database ~s.", [Sig, DbName]), [{_, Pid}] = ets:lookup(group_servers_by_sig, {DbName, Sig}), - exit(Pid, kill), - receive {'EXIT', Pid, _} -> - delete_from_ets(Pid, DbName, Sig) - end + couch_util:shutdown_sync(Pid), + delete_from_ets(Pid, DbName, Sig) end, Names), delete_index_dir(Root, DbName), - file:delete(Root ++ "/." ++ binary_to_list(DbName) ++ "_temp"), - {noreply, Server}. + RootDelDir = couch_config:get("couchdb", "view_index_dir"), + couch_file:delete(RootDelDir, Root ++ "/." ++ ?b2l(DbName) ++ "_temp"). handle_info({'EXIT', FromPid, Reason}, Server) -> case ets:lookup(couch_groups_by_updater, FromPid) of @@ -336,19 +346,19 @@ code_change(_OldVsn, State, _Extra) -> delete_index_dir(RootDir, DbName) -> - nuke_dir(RootDir ++ "/." ++ ?b2l(DbName) ++ "_design"). + nuke_dir(RootDir, RootDir ++ "/." ++ ?b2l(DbName) ++ "_design"). -nuke_dir(Dir) -> +nuke_dir(RootDelDir, Dir) -> case file:list_dir(Dir) of {error, enoent} -> ok; % doesn't exist {ok, Files} -> lists:foreach( fun(File)-> Full = Dir ++ "/" ++ File, - case file:delete(Full) of + case couch_file:delete(RootDelDir, Full, false) of ok -> ok; {error, eperm} -> - ok = nuke_dir(Full) + ok = nuke_dir(RootDelDir, Full) end end, Files), @@ -358,81 +368,67 @@ nuke_dir(Dir) -> % keys come back in the language of btree - tuples. less_json_ids({JsonA, IdA}, {JsonB, IdB}) -> - case JsonA == JsonB of - false -> - less_json(JsonA, JsonB); - true -> - IdA < IdB - end. - - -less_json(A, B) -> - TypeA = type_sort(A), - TypeB = type_sort(B), - if TypeA == TypeB -> - less_same_type(A, B); - true -> - TypeA < TypeB + case less_json0(JsonA, JsonB) of + 0 -> + IdA < IdB; + Result -> + Result < 0 end. -type_sort(V) when is_atom(V) -> 0; -type_sort(V) when is_integer(V) -> 1; -type_sort(V) when is_float(V) -> 1; -type_sort(V) when is_binary(V) -> 2; -type_sort(V) when is_list(V) -> 3; -type_sort({V}) when is_list(V) -> 4; -type_sort(V) when is_tuple(V) -> 5. +less_json(A,B) -> + less_json0(A,B) < 0. + +less_json0(A,A) -> 0; + +less_json0(A,B) when is_atom(A), is_atom(B) -> atom_sort(A) - atom_sort(B); +less_json0(A,_) when is_atom(A) -> -1; +less_json0(_,B) when is_atom(B) -> 1; + +less_json0(A,B) when is_number(A), is_number(B) -> A - B; +less_json0(A,_) when is_number(A) -> -1; +less_json0(_,B) when is_number(B) -> 1; +less_json0(A,B) when is_binary(A), is_binary(B) -> couch_util:collate(A,B); +less_json0(A,_) when is_binary(A) -> -1; +less_json0(_,B) when is_binary(B) -> 1; + +less_json0(A,B) when is_list(A), is_list(B) -> less_list(A,B); +less_json0(A,_) when is_list(A) -> -1; +less_json0(_,B) when is_list(B) -> 1; + +less_json0({A},{B}) when is_list(A), is_list(B) -> less_props(A,B); +less_json0({A},_) when is_list(A) -> -1; +less_json0(_,{B}) when is_list(B) -> 1. atom_sort(null) -> 1; atom_sort(false) -> 2; atom_sort(true) -> 3. - -less_same_type(A,B) when is_atom(A) -> - atom_sort(A) < atom_sort(B); -less_same_type(A,B) when is_binary(A) -> - couch_util:collate(A, B) < 0; -less_same_type({AProps}, {BProps}) -> - less_props(AProps, BProps); -less_same_type(A, B) when is_list(A) -> - less_list(A, B); -less_same_type(A, B) -> - A < B. - less_props([], [_|_]) -> - true; + -1; less_props(_, []) -> - false; + 1; less_props([{AKey, AValue}|RestA], [{BKey, BValue}|RestB]) -> case couch_util:collate(AKey, BKey) of - -1 -> true; - 1 -> false; 0 -> - case less_json(AValue, BValue) of - true -> true; - false -> - case less_json(BValue, AValue) of - true -> false; - false -> - less_props(RestA, RestB) - end - end + case less_json0(AValue, BValue) of + 0 -> + less_props(RestA, RestB); + Result -> + Result + end; + Result -> + Result end. less_list([], [_|_]) -> - true; + -1; less_list(_, []) -> - false; + 1; less_list([A|RestA], [B|RestB]) -> - case less_json(A,B) of - true -> true; - false -> - case less_json(B,A) of - true -> false; - false -> - less_list(RestA, RestB) - end + case less_json0(A,B) of + 0 -> + less_list(RestA, RestB); + Result -> + Result end. - - diff --git a/src/couchdb/couch_view_group.erl b/src/couchdb/couch_view_group.erl index 6589bc6a..aec52543 100644 --- a/src/couchdb/couch_view_group.erl +++ b/src/couchdb/couch_view_group.erl @@ -32,7 +32,8 @@ compactor_pid=nil, waiting_commit=false, waiting_list=[], - ref_counter=nil + ref_counter=nil, + db_update_notifier=nil }). % api methods @@ -75,20 +76,31 @@ start_link(InitArgs) -> end. % init creates a closure which spawns the appropriate view_updater. -init({InitArgs, ReturnPid, Ref}) -> +init({{_, DbName, _} = InitArgs, ReturnPid, Ref}) -> process_flag(trap_exit, true), case prepare_group(InitArgs, false) of - {ok, #group{db=Db, fd=Fd}=Group} -> - couch_db:monitor(Db), - Owner = self(), - Pid = spawn_link(fun()-> couch_view_updater:update(Owner, Group) end), - {ok, RefCounter} = couch_ref_counter:start([Fd]), - {ok, #group_state{ - db_name=couch_db:name(Db), - init_args=InitArgs, - updater_pid = Pid, - group=Group, - ref_counter=RefCounter}}; + {ok, #group{db=Db, fd=Fd, current_seq=Seq}=Group} -> + case Seq > couch_db:get_update_seq(Db) of + true -> + ReturnPid ! {Ref, self(), {error, invalid_view_seq}}, + ignore; + _ -> + couch_db:monitor(Db), + {ok, RefCounter} = couch_ref_counter:start([Fd]), + Server = self(), + {ok, Notifier} = couch_db_update_notifier:start_link( + fun({compacted, DbName1}) when DbName1 =:= DbName -> + ok = gen_server:cast(Server, reopen_db); + (_) -> + ok + end), + {ok, #group_state{ + db_update_notifier=Notifier, + db_name=couch_db:name(Db), + init_args=InitArgs, + group=Group, + ref_counter=RefCounter}} + end; Error -> ReturnPid ! {Ref, self(), Error}, ignore @@ -113,11 +125,11 @@ init({InitArgs, ReturnPid, Ref}) -> handle_call({request_group, RequestSeq}, From, #group_state{ db_name=DbName, - group=#group{current_seq=Seq}=Group, + group=#group{current_seq=Seq, db=OldDb}=Group, updater_pid=nil, waiting_list=WaitList }=State) when RequestSeq > Seq -> - {ok, Db} = couch_db:open(DbName, []), + {ok, Db} = reopen_db(DbName, OldDb), Group2 = Group#group{db=Db}, Owner = self(), Pid = spawn_link(fun()-> couch_view_updater:update(Owner, Group2) end), @@ -152,11 +164,11 @@ handle_call(request_group_info, _From, State) -> handle_cast({start_compact, CompactFun}, #group_state{compactor_pid=nil} = State) -> #group_state{ - group = #group{name = GroupId, sig = GroupSig} = Group, + group = #group{name = GroupId, sig = GroupSig, db = OldDb} = Group, init_args = {RootDir, DbName, _} } = State, ?LOG_INFO("View index compaction starting for ~s ~s", [DbName, GroupId]), - {ok, Db} = couch_db:open(DbName, []), + {ok, Db} = reopen_db(DbName, OldDb), {ok, Fd} = open_index_file(compact, RootDir, DbName, GroupSig), NewGroup = reset_file(Db, Fd, DbName, Group), Pid = spawn_link(fun() -> CompactFun(Group, NewGroup) end), @@ -170,15 +182,16 @@ handle_cast({compact_done, #group{current_seq=NewSeq} = NewGroup}, when NewSeq >= OldSeq -> #group_state{ group = #group{name=GroupId, fd=OldFd, sig=GroupSig} = Group, - init_args = {RootDir, DbName, _}, + init_args = {RootDir, DbName, _}, updater_pid = UpdaterPid, + compactor_pid = CompactorPid, ref_counter = RefCounter } = State, ?LOG_INFO("View index compaction complete for ~s ~s", [DbName, GroupId]), FileName = index_file_name(RootDir, DbName, GroupSig), CompactName = index_file_name(compact, RootDir, DbName, GroupSig), - file:delete(FileName), + ok = couch_file:delete(RootDir, FileName), ok = file:rename(CompactName, FileName), %% if an updater is running, kill it and start a new one @@ -193,6 +206,8 @@ handle_cast({compact_done, #group{current_seq=NewSeq} = NewGroup}, end, %% cleanup old group + unlink(CompactorPid), + receive {'EXIT', CompactorPid, normal} -> ok after 0 -> ok end, unlink(OldFd), couch_ref_counter:drop(RefCounter), {ok, NewRefCounter} = couch_ref_counter:start([NewGroup#group.fd]), @@ -216,13 +231,14 @@ handle_cast({compact_done, NewGroup}, State) -> ?LOG_INFO("View index compaction still behind for ~s ~s -- current: ~p " ++ "compact: ~p", [DbName, GroupId, CurrentSeq, NewGroup#group.current_seq]), couch_db:close(NewGroup#group.db), - {ok, Db} = couch_db:open(DbName, []), Pid = spawn_link(fun() -> + {ok, Db} = couch_db:open_int(DbName, []), {_,Ref} = erlang:spawn_monitor(fun() -> couch_view_updater:update(nil, NewGroup#group{db = Db}) end), receive {'DOWN', Ref, _, _, {new_group, NewGroup2}} -> + couch_db:close(Db), #group{name=GroupId} = NewGroup2, Pid2 = couch_view:get_group_server(DbName, GroupId), gen_server:cast(Pid2, {compact_done, NewGroup2}) @@ -246,10 +262,14 @@ handle_cast({partial_update, Pid, NewGroup}, #group_state{updater_pid=Pid} {noreply, State#group_state{group=NewGroup, waiting_commit=true}}; handle_cast({partial_update, _, _}, State) -> %% message from an old (probably pre-compaction) updater; ignore - {noreply, State}. + {noreply, State}; + +handle_cast(reopen_db, #group_state{group = Group, db_name = DbName} = State) -> + {ok, Db} = reopen_db(DbName, Group#group.db), + {noreply, State#group_state{group = Group#group{db = Db}}}. handle_info(delayed_commit, #group_state{db_name=DbName,group=Group}=State) -> - {ok, Db} = couch_db:open(DbName, []), + {ok, Db} = couch_db:open_int(DbName, []), CommittedSeq = couch_db:get_committed_update_seq(Db), couch_db:close(Db), if CommittedSeq >= Group#group.current_seq -> @@ -284,7 +304,7 @@ handle_info({'EXIT', FromPid, {new_group, #group{db=Db}=Group}}, group=Group#group{db=nil}, updater_pid=nil}}; StillWaiting -> % we still have some waiters, reopen the database and reupdate the index - {ok, Db2} = couch_db:open(DbName, []), + {ok, Db2} = couch_db:open_int(DbName, []), Group2 = Group#group{db=Db2}, Owner = self(), Pid = spawn_link(fun() -> couch_view_updater:update(Owner, Group2) end), @@ -332,9 +352,10 @@ handle_info({'DOWN',_,_,_,_}, State) -> terminate(Reason, #group_state{updater_pid=Update, compactor_pid=Compact}=S) -> + couch_db_update_notifier:stop(S#group_state.db_update_notifier), reply_all(S, Reason), - catch exit(Update, Reason), - catch exit(Compact, Reason), + couch_util:shutdown_sync(Update), + couch_util:shutdown_sync(Compact), ok. code_change(_OldVsn, State, _Extra) -> @@ -363,8 +384,8 @@ reply_all(#group_state{waiting_list=WaitList}=State, Reply) -> [catch gen_server:reply(Pid, Reply) || {Pid, _} <- WaitList], State#group_state{waiting_list=[]}. -prepare_group({RootDir, DbName, #group{sig=Sig}=Group}, ForceReset)-> - case couch_db:open(DbName, []) of +prepare_group({RootDir, DbName, #group{sig=Sig, db=OldDb}=Group}, ForceReset)-> + case reopen_db(DbName, OldDb) of {ok, Db} -> case open_index_file(RootDir, DbName, Sig) of {ok, Fd} -> @@ -429,7 +450,7 @@ open_index_file(compact, RootDir, DbName, GroupSig) -> end. open_temp_group(DbName, Language, DesignOptions, MapSrc, RedSrc) -> - case couch_db:open(DbName, []) of + case couch_db:open_int(DbName, []) of {ok, Db} -> View = #view{map_names=[<<"_temp">>], id_num=0, @@ -437,8 +458,8 @@ open_temp_group(DbName, Language, DesignOptions, MapSrc, RedSrc) -> def=MapSrc, reduce_funs= if RedSrc==[] -> []; true -> [{<<"_temp">>, RedSrc}] end, options=DesignOptions}, - - {ok, Db, set_view_sig(#group{name = <<"_temp">>, db=Db, views=[View], + couch_db:close(Db), + {ok, set_view_sig(#group{name = <<"_temp">>, views=[View], def_lang=Language, design_options=DesignOptions})}; Error -> Error @@ -448,14 +469,15 @@ set_view_sig(#group{ views=Views, def_lang=Language, design_options=DesignOptions}=G) -> - G#group{sig=erlang:md5(term_to_binary({Views, Language, DesignOptions}))}. + G#group{sig=couch_util:md5(term_to_binary({Views, Language, DesignOptions}))}. open_db_group(DbName, GroupId) -> - case couch_db:open(DbName, []) of + case couch_db:open_int(DbName, []) of {ok, Db} -> case couch_db:open_doc(Db, GroupId) of {ok, Doc} -> - {ok, Db, design_doc_to_view_group(Doc)}; + couch_db:close(Db), + {ok, design_doc_to_view_group(Doc)}; Else -> couch_db:close(Db), Else @@ -494,16 +516,16 @@ get_group_info(State) -> % maybe move to another module design_doc_to_view_group(#doc{id=Id,body={Fields}}) -> - Language = proplists:get_value(<<"language">>, Fields, <<"javascript">>), - {DesignOptions} = proplists:get_value(<<"options">>, Fields, {[]}), - {RawViews} = proplists:get_value(<<"views">>, Fields, {[]}), + Language = couch_util:get_value(<<"language">>, Fields, <<"javascript">>), + {DesignOptions} = couch_util:get_value(<<"options">>, Fields, {[]}), + {RawViews} = couch_util:get_value(<<"views">>, Fields, {[]}), % add the views to a dictionary object, with the map source as the key DictBySrc = lists:foldl( fun({Name, {MRFuns}}, DictBySrcAcc) -> - MapSrc = proplists:get_value(<<"map">>, MRFuns), - RedSrc = proplists:get_value(<<"reduce">>, MRFuns, null), - {ViewOptions} = proplists:get_value(<<"options">>, MRFuns, {[]}), + MapSrc = couch_util:get_value(<<"map">>, MRFuns), + RedSrc = couch_util:get_value(<<"reduce">>, MRFuns, null), + {ViewOptions} = couch_util:get_value(<<"options">>, MRFuns, {[]}), View = case dict:find({MapSrc, ViewOptions}, DictBySrcAcc) of {ok, View0} -> View0; @@ -537,7 +559,7 @@ reset_file(Db, Fd, DbName, #group{sig=Sig,name=Name} = Group) -> init_group(Db, Fd, reset_group(Group), nil). delete_index_file(RootDir, DbName, GroupSig) -> - file:delete(index_file_name(RootDir, DbName, GroupSig)). + couch_file:delete(RootDir, index_file_name(RootDir, DbName, GroupSig)). init_group(Db, Fd, #group{views=Views}=Group, nil) -> init_group(Db, Fd, Group, @@ -566,7 +588,7 @@ init_group(Db, Fd, #group{def_lang=Lang,views=Views}= {Count, Reduced} end, - case proplists:get_value(<<"collation">>, Options, <<"default">>) of + case couch_util:get_value(<<"collation">>, Options, <<"default">>) of <<"default">> -> Less = fun couch_view:less_json_ids/2; <<"raw">> -> @@ -581,4 +603,7 @@ init_group(Db, Fd, #group{def_lang=Lang,views=Views}= Group#group{db=Db, fd=Fd, current_seq=Seq, purge_seq=PurgeSeq, id_btree=IdBtree, views=Views2}. - +reopen_db(DbName, nil) -> + couch_db:open_int(DbName, []); +reopen_db(_DbName, Db) -> + couch_db:reopen(Db). diff --git a/src/couchdb/couch_view_updater.erl b/src/couchdb/couch_view_updater.erl index aa7d98fc..2a9c960f 100644 --- a/src/couchdb/couch_view_updater.erl +++ b/src/couchdb/couch_view_updater.erl @@ -49,9 +49,9 @@ update(Owner, Group) -> % update status every half second couch_task_status:set_update_frequency(500), #group{ design_options = DesignOptions } = Group, - IncludeDesign = proplists:get_value(<<"include_design">>, + IncludeDesign = couch_util:get_value(<<"include_design">>, DesignOptions, false), - LocalSeq = proplists:get_value(<<"local_seq">>, DesignOptions, false), + LocalSeq = couch_util:get_value(<<"local_seq">>, DesignOptions, false), DocOpts = case LocalSeq of true -> [conflicts, deleted_conflicts, local_seq]; diff --git a/src/couchdb/couch_work_queue.erl b/src/couchdb/couch_work_queue.erl index ca9445d3..decfcad8 100644 --- a/src/couchdb/couch_work_queue.erl +++ b/src/couchdb/couch_work_queue.erl @@ -13,11 +13,11 @@ -module(couch_work_queue). -behaviour(gen_server). --export([new/2,queue/2,dequeue/1,close/1]). +-export([new/2,queue/2,dequeue/1,dequeue/2,close/1]). -export([init/1, terminate/2, handle_call/3, handle_cast/2, code_change/3, handle_info/2]). -record(q, { - buffer=[], + queue=queue:new(), blocked=[], max_size, max_items, @@ -34,7 +34,10 @@ queue(Wq, Item) -> gen_server:call(Wq, {queue, Item}, infinity). dequeue(Wq) -> - try gen_server:call(Wq, dequeue, infinity) + dequeue(Wq, all). + +dequeue(Wq, MaxItems) -> + try gen_server:call(Wq, {dequeue, MaxItems}, infinity) catch _:_ -> closed end. @@ -48,13 +51,13 @@ init({MaxSize,MaxItems}) -> terminate(_Reason, #q{work_waiter=nil}) -> ok; -terminate(_Reason, #q{work_waiter=WW}) -> - gen_server:reply(WW, closed). +terminate(_Reason, #q{work_waiter={WWFrom, _}}) -> + gen_server:reply(WWFrom, closed). handle_call({queue, Item}, From, #q{work_waiter=nil}=Q0) -> Q = Q0#q{size=Q0#q.size + byte_size(term_to_binary(Item)), items=Q0#q.items + 1, - buffer=[Item | Q0#q.buffer]}, + queue=queue:in(Item, Q0#q.queue)}, case (Q#q.size >= Q#q.max_size) orelse (Q#q.items >= Q#q.max_items) of true -> @@ -62,24 +65,44 @@ handle_call({queue, Item}, From, #q{work_waiter=nil}=Q0) -> false -> {reply, ok, Q} end; -handle_call({queue, Item}, _From, #q{work_waiter=WW}=Q) -> - gen_server:reply(WW, {ok, [Item]}), +handle_call({queue, Item}, _From, #q{work_waiter={WWFrom, _Max}}=Q) -> + gen_server:reply(WWFrom, {ok, [Item]}), {reply, ok, Q#q{work_waiter=nil}}; -handle_call(dequeue, _From, #q{work_waiter=WW}) when WW /= nil -> +handle_call({dequeue, _Max}, _From, #q{work_waiter=WW}) when WW /= nil -> exit("Only one caller allowed to wait for work at a time"); -handle_call(dequeue, From, #q{items=0}=Q) -> - {noreply, Q#q{work_waiter=From}}; -handle_call(dequeue, _From, #q{buffer=Buff, max_size=MaxSize, - max_items=MaxItems, close_on_dequeue=Close}=Q) -> - [gen_server:reply(From, ok) || From <- Q#q.blocked], - Q2 = #q{max_size=MaxSize, max_items=MaxItems}, - if Close -> - {stop, normal, {ok, Buff}, Q2}; +handle_call({dequeue, Max}, From, #q{items=0}=Q) -> + {noreply, Q#q{work_waiter={From, Max}}}; +handle_call({dequeue, Max}, _From, #q{queue=Queue, max_size=MaxSize, + max_items=MaxItems, items=Items,close_on_dequeue=Close}=Q) -> + if Max >= Items orelse Max == all -> + [gen_server:reply(From, ok) || From <- Q#q.blocked], + Q2 = #q{max_size=MaxSize, max_items=MaxItems}, + if Close -> + {stop, normal, {ok, queue:to_list(Queue)}, Q2}; + true -> + {reply, {ok, queue:to_list(Queue)}, Q2} + end; true -> - {reply, {ok, Buff}, #q{max_size=MaxSize, max_items=MaxItems}} + {DequeuedItems, Queue2, Blocked2} = + dequeue_items(Max, Queue, Q#q.blocked, []), + {reply, {ok, DequeuedItems}, + Q#q{items=Items-Max,blocked=Blocked2,queue=Queue2}} end. -handle_cast(close, #q{buffer=[]}=Q) -> +dequeue_items(0, Queue, Blocked, DequeuedAcc) -> + {lists:reverse(DequeuedAcc), Queue, Blocked}; +dequeue_items(NumItems, Queue, Blocked, DequeuedAcc) -> + {{value, Item}, Queue2} = queue:out(Queue), + case Blocked of + [] -> + Blocked2 = Blocked; + [From|Blocked2] -> + gen_server:reply(From, ok) + end, + dequeue_items(NumItems-1, Queue2, Blocked2, [Item | DequeuedAcc]). + + +handle_cast(close, #q{items=0}=Q) -> {stop, normal, Q}; handle_cast(close, Q) -> {noreply, Q#q{close_on_dequeue=true}}. diff --git a/src/couchdb/priv/Makefile.am b/src/couchdb/priv/Makefile.am index 5b76f8cd..b36d828d 100644 --- a/src/couchdb/priv/Makefile.am +++ b/src/couchdb/priv/Makefile.am @@ -47,7 +47,7 @@ COUCHJS_SRCS = \ locallibbin_PROGRAMS = couchjs couchjs_SOURCES = $(COUCHJS_SRCS) couchjs_LDFLAGS = $(CURL_LDFLAGS) -couchjs_CFLAGS = $(CURL_CFLAGS) +couchjs_CFLAGS = -D_BSD_SOURCE $(CURL_CFLAGS) couchjs_LDADD = $(CURL_LDFLAGS) @JSLIB@ couchpriv_DATA = stat_descriptions.cfg diff --git a/src/couchdb/priv/couch_js/http.c b/src/couchdb/priv/couch_js/http.c index 998c2439..6c2a8a82 100644 --- a/src/couchdb/priv/couch_js/http.c +++ b/src/couchdb/priv/couch_js/http.c @@ -10,6 +10,7 @@ // License for the specific language governing permissions and limitations under // the License. +#include <stdio.h> #include <stdlib.h> #include <string.h> #include <jsapi.h> @@ -403,7 +404,8 @@ go(JSContext* cx, JSObject* obj, HTTPData* http, char* body, size_t bodylen) { HTTP_HANDLE = curl_easy_init(); curl_easy_setopt(HTTP_HANDLE, CURLOPT_READFUNCTION, send_body); - curl_easy_setopt(HTTP_HANDLE, CURLOPT_SEEKFUNCTION, seek_body); + curl_easy_setopt(HTTP_HANDLE, CURLOPT_SEEKFUNCTION, + (curl_seek_callback) seek_body); curl_easy_setopt(HTTP_HANDLE, CURLOPT_HEADERFUNCTION, recv_header); curl_easy_setopt(HTTP_HANDLE, CURLOPT_WRITEFUNCTION, recv_body); curl_easy_setopt(HTTP_HANDLE, CURLOPT_NOPROGRESS, 1); diff --git a/src/couchdb/priv/stat_descriptions.cfg.in b/src/couchdb/priv/stat_descriptions.cfg.in index 554124cf..b80d7684 100644 --- a/src/couchdb/priv/stat_descriptions.cfg.in +++ b/src/couchdb/priv/stat_descriptions.cfg.in @@ -1,3 +1,15 @@ +%% 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. + % Style guide for descriptions: Start with a lowercase letter & do not add % a trailing full-stop / period % Please keep this in alphabetical order @@ -7,6 +19,8 @@ {couchdb, open_databases, "number of open databases"}. {couchdb, open_os_files, "number of file descriptors CouchDB has open"}. {couchdb, request_time, "length of a request inside CouchDB without MochiWeb"}. +{couchdb, auth_cache_hits, "number of authentication cache hits"}. +{couchdb, auth_cache_misses, "number of authentication cache misses"}. {httpd, bulk_requests, "number of bulk requests"}. {httpd, requests, "number of HTTP requests"}. @@ -18,7 +32,6 @@ {httpd_request_methods, 'DELETE', "number of HTTP DELETE requests"}. {httpd_request_methods, 'GET', "number of HTTP GET requests"}. {httpd_request_methods, 'HEAD', "number of HTTP HEAD requests"}. -{httpd_request_methods, 'MOVE', "number of HTTP MOVE requests"}. {httpd_request_methods, 'POST', "number of HTTP POST requests"}. {httpd_request_methods, 'PUT', "number of HTTP PUT requests"}. |