diff options
75 files changed, 1409 insertions, 730 deletions
diff --git a/apps/couch/CHANGES b/apps/couch/CHANGES index 08458f10..fdfd31d2 100644 --- a/apps/couch/CHANGES +++ b/apps/couch/CHANGES @@ -1,6 +1,65 @@ Apache CouchDB CHANGES ====================== +Version 1.0.2 +------------- + +Futon: + + * Make test suite work with Safari and Chrome. + * Fixed animated progress spinner. + * Fix raw view document link due to overzealous URI encoding. + * Spell javascript correctly in loadScript(uri). + +Storage System: + + * Fix leaking file handles after compacting databases and views. + * Fix databases forgetting their validation function after compaction. + * Fix occasional timeout errors after successfully compacting large databases. + * Fix ocassional error when writing to a database that has just been compacted. + * Fix occasional timeout errors on systems with slow or heavily loaded IO. + * Fix for OOME when compactions include documents with many conflicts. + * Fix for missing attachment compression when MIME types included parameters. + * Preserve purge metadata during compaction to avoid spurious view rebuilds. + * Fix spurious conflicts introduced when uploading an attachment after + a doc has been in a conflict. See COUCHDB-902 for details. + * Fix for frequently edited documents in multi-master deployments being + duplicated in _changes and _all_docs. See COUCHDDB-968 for details on how + to repair. + * Significantly higher read and write throughput against database and + view index files. + +Log System: + + * Reduce lengthy stack traces. + * Allow logging of native <xml> types. + +HTTP Interface: + + * Allow reduce=false parameter in map-only views. + * Fix parsing of Accept headers. + * Fix for multipart GET APIs when an attachment was created during a + local-local replication. See COUCHDB-1022 for details. + +Replicator: + + * Updated ibrowse library to 2.1.2 fixing numerous replication issues. + * Make sure that the replicator respects HTTP settings defined in the config. + * Fix error when the ibrowse connection closes unexpectedly. + * Fix authenticated replication (with HTTP basic auth) of design documents + with attachments. + * Various fixes to make replication more resilient for edge-cases. + +View Server: + + * Don't trigger view updates when requesting `_design/doc/_info`. + * Fix for circular references in CommonJS requires. + * Made isArray() function available to functions executed in the query server. + * Documents are now sealed before being passed to map functions. + * Force view compaction failure when duplicated document data exists. When + this error is seen in the logs users should rebuild their views from + scratch to fix the issue. See COUCHDB-999 for details. + Version 1.0.1 ------------- @@ -11,7 +70,7 @@ Storage System: Replicator: - * Added support for replication via an HTTP/HTTP proxy. + * Added support for replication via an HTTP/HTTPS proxy. * Fix pull replication of attachments from 0.11 to 1.0.x. * Make the _changes feed work with non-integer seqnums. diff --git a/apps/couch/NEWS b/apps/couch/NEWS index bd6b5d2b..9550856e 100644 --- a/apps/couch/NEWS +++ b/apps/couch/NEWS @@ -7,12 +7,34 @@ For details about backwards incompatible changes, see: Each release section notes when backwards incompatible changes have been made. +Version 1.0.2 +------------- + + * Make test suite work with Safari and Chrome. + * Fix leaking file handles after compacting databases and views. + * Fix databases forgetting their validation function after compaction. + * Fix occasional timeout errors. + * Reduce lengthy stack traces. + * Allow logging of native <xml> types. + * Updated ibrowse library to 2.1.2 fixing numerous replication issues. + * Fix authenticated replication of design documents with attachments. + * Fix multipart GET APIs by always sending attachments in compressed + form when the source attachment is compressed on disk. Fixes a possible + edge case when an attachment underwent local-local replication. + * Various fixes to make replicated more resilient for edge-cases. + * Don't trigger a view update when requesting `_design/doc/_info`. + * Fix for circular references in CommonJS requires. + * Fix for frequently edited documents in multi-master deployments being + duplicated in _changes and _all_docs. + * Fix spurious conflict generation during attachment uploads. + * Fix for various bugs in Futon. + Version 1.0.1 ------------- * Fix data corruption bug COUCHDB-844. Please see http://couchdb.apache.org/notice/1.0.1.html for details. - * Added support for replication via an HTTP/HTTP proxy. + * Added support for replication via an HTTP/HTTPS proxy. * Fixed various replicator bugs for interop with older CouchDB versions. * Show fields saved along with _deleted=true. Allows for auditing of deletes. * Enable basic-auth popup when required to access the server, to prevent @@ -38,6 +60,7 @@ Version 0.11.2 * User documents can now be deleted by admins or the user. * Avoid potential DOS attack by guarding all creation of atoms. * Some Futon and JavaScript library bugfixes. + * Fixed CVE-2010-2234: Apache CouchDB Cross Site Request Forgery Attack Version 0.11.1 -------------- diff --git a/apps/couch/THANKS b/apps/couch/THANKS index ebee4845..15072e2a 100644 --- a/apps/couch/THANKS +++ b/apps/couch/THANKS @@ -65,5 +65,10 @@ suggesting improvements or submitting changes. Some of these people are: * Juhani Ränkimies <juhani@juranki.com> * Lim Yue Chuan <shasderias@gmail.com> * David Davis <xantus@xantus.org> + * Dale Harvey <dale@arandomurl.com> + * Juuso Väänänen <juuso@vaananen.org> + * Benjamin Young <byoung@bigbluehat.com> + * Gabriel Farrell <gsf747@gmail.com> + * Mike Leddy <mike@loop.com.br> For a list of authors see the `AUTHORS` file. diff --git a/apps/couch/include/couch_db.hrl b/apps/couch/include/couch_db.hrl index 12b0ac90..08c89032 100644 --- a/apps/couch/include/couch_db.hrl +++ b/apps/couch/include/couch_db.hrl @@ -266,13 +266,7 @@ body = nil, options = [ {response_format,binary}, - {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") - )} + {inactivity_timeout, 30000} ], retries = 10, pause = 500, diff --git a/apps/couch/priv/stat_descriptions.cfg b/apps/couch/priv/stat_descriptions.cfg index 5c972ddf..b80d7684 100644 --- a/apps/couch/priv/stat_descriptions.cfg +++ b/apps/couch/priv/stat_descriptions.cfg @@ -32,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"}. diff --git a/apps/couch/src/couch_auth_cache.erl b/apps/couch/src/couch_auth_cache.erl index 0264a69d..8b911543 100644 --- a/apps/couch/src/couch_auth_cache.erl +++ b/apps/couch/src/couch_auth_cache.erl @@ -135,6 +135,7 @@ handle_db_event({Event, DbName}) -> 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 -> @@ -158,6 +159,14 @@ handle_call(auth_db_created, _From, 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 -> @@ -175,7 +184,7 @@ handle_call({new_max_cache_size, NewSize}, _From, State) -> end, NewState = State#state{ max_cache_size = NewSize, - cache_size = erlang:min(NewSize, State#state.cache_size) + cache_size = lists:min([NewSize, State#state.cache_size]) }, {reply, ok, NewState}; @@ -338,7 +347,7 @@ cache_needs_refresh() -> reopen_auth_db(AuthDb) -> - case (catch gen_server:call(AuthDb#db.main_pid, get_db, infinity)) of + case (catch couch_db:reopen(AuthDb)) of {ok, AuthDb2} -> AuthDb2; _ -> diff --git a/apps/couch/src/couch_db.erl b/apps/couch/src/couch_db.erl index 0e42980e..7293b9bb 100644 --- a/apps/couch/src/couch_db.erl +++ b/apps/couch/src/couch_db.erl @@ -25,6 +25,7 @@ -export([set_security/2,get_security/1]). -export([changes_since/5,changes_since/6,read_doc/2,new_revid/1]). -export([check_is_admin/1, check_is_reader/1, get_doc_count/1, load_validation_funs/1]). +-export([reopen/1]). -include("couch_db.hrl"). @@ -84,6 +85,17 @@ open(DbName, Options) -> Else -> Else end. +reopen(#db{main_pid = Pid, fd = Fd, fd_monitor = OldRef, user_ctx = UserCtx}) -> + {ok, #db{fd = NewFd} = NewDb} = gen_server:call(Pid, get_db, infinity), + case NewFd =:= Fd of + true -> + {ok, NewDb#db{user_ctx = UserCtx}}; + false -> + erlang:demonitor(OldRef), + NewRef = erlang:monitor(process, NewFd), + {ok, NewDb#db{user_ctx = UserCtx, fd_monitor = NewRef}} + end. + ensure_full_commit(#db{main_pid=Pid, instance_start_time=StartTime}) -> ok = gen_server:call(Pid, full_commit, infinity), {ok, StartTime}. @@ -584,7 +596,8 @@ 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), Db#db.revs_limit), NewTree end, OldTree, Bucket), @@ -845,11 +858,12 @@ 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) -> +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(IdentityMd5, InMd5), - Att#att{data={Fd, NewStreamData}, md5=Md5, att_len=Len, disk_len=Len}; + 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) -> @@ -947,7 +961,7 @@ with_stream(Fd, #att{md5=InMd5,type=Type,encoding=Enc}=Att, Fun) -> write_streamed_attachment(_Stream, _F, 0) -> ok; -write_streamed_attachment(Stream, F, LenLeft) -> +write_streamed_attachment(Stream, F, LenLeft) when LenLeft > 0 -> Bin = F(), ok = couch_stream:write(Stream, Bin), write_streamed_attachment(Stream, F, LenLeft - size(Bin)). diff --git a/apps/couch/src/couch_db_updater.erl b/apps/couch/src/couch_db_updater.erl index ab078caf..835d188c 100644 --- a/apps/couch/src/couch_db_updater.erl +++ b/apps/couch/src/couch_db_updater.erl @@ -204,9 +204,11 @@ handle_cast({compact_done, CompactFilepath}, #db{filepath=Filepath}=Db) -> couch_file:delete(RootDir, Filepath), ok = file:rename(CompactFilepath, Filepath), close_db(Db), - ok = gen_server:call(couch_server, {db_updated, NewDb2}, infinity), + NewDb3 = refresh_validate_doc_funs(NewDb2), + ok = gen_server:call(couch_server, {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 for ~s still behind main file " "(update seq=~p. compact update seq=~p). Retrying.", @@ -518,16 +520,17 @@ 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( 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), + Limit) of {_NewTree, conflicts} when (not OldDeleted) -> send_result(Client, Id, {Pos-1,PrevRevs}, conflict), AccTree; @@ -558,7 +561,7 @@ merge_rev_trees(MergeConflicts, [NewDocs|RestDocsList], 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), Limit), % we changed the rev id, this tells the caller we did send_result(Client, Id, {Pos-1,PrevRevs}, {ok, {OldPos + 1, NewRevId}}), @@ -572,15 +575,15 @@ 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), Limit), NewTree end end, OldTree, NewDocs), 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}, @@ -588,8 +591,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. @@ -609,7 +612,8 @@ update_docs_int(Db, DocsList, NonRepDocs, MergeConflicts, FullCommit) -> #db{ id_tree = DocInfoByIdBTree, seq_tree = 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. @@ -622,11 +626,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), @@ -794,15 +796,6 @@ copy_doc_attachments(#db{fd=SrcFd}=SrcDb, {Pos,_RevId}, SrcSp, DestFd) -> end, BinInfos), {BodyData, NewBinInfos}. -copy_rev_tree_attachments(SrcDb, DestFd, Tree) -> - couch_key_tree:map( - fun(Rev, {IsDel, Sp, Seq}, leaf) -> - DocBody = copy_doc_attachments(SrcDb, Rev, Sp, DestFd), - {IsDel, DocBody, Seq}; - (_, _, branch) -> - ?REV_MISSING - end, Tree). - merge_lookups(Infos, []) -> Infos; merge_lookups([], _) -> @@ -816,20 +809,19 @@ copy_docs(Db, #db{fd=DestFd}=NewDb, MixedInfos, Retry) -> % lookup any necessary full_doc_infos DocInfoIds = [Id || #doc_info{id=Id} <- MixedInfos], LookupResults = couch_btree:lookup(Db#db.id_tree, DocInfoIds), - Infos = merge_lookups(MixedInfos, LookupResults), + % COUCHDB-968, make sure we prune duplicates during compaction + Infos = lists:usort(fun(#full_doc_info{id=A}, #full_doc_info{id=B}) -> + A =< B + end, merge_lookups(MixedInfos, LookupResults)), - % write out the attachments - NewInfos0 = [Info#full_doc_info{rev_tree=copy_rev_tree_attachments(Db, - DestFd, RevTree)} || #full_doc_info{rev_tree=RevTree}=Info <- Infos], - - % write out the docs - % we do this in 2 stages so the docs are written out contiguously, making - % view indexing and replication faster. - NewInfos1 = [Info#full_doc_info{rev_tree=couch_key_tree:map_leafs( - fun(_Key, {IsDel, DocBody, Seq}) -> + NewInfos1 = [Info#full_doc_info{rev_tree=couch_key_tree:map( + fun(Rev, {IsDel, Sp, Seq}, leaf) -> + DocBody = copy_doc_attachments(Db, Rev, Sp, DestFd), {ok, Pos} = couch_file:append_term_md5(DestFd, DocBody), - {IsDel, Pos, Seq} - end, RevTree)} || #full_doc_info{rev_tree=RevTree}=Info <- NewInfos0], + {IsDel, Pos, Seq}; + (_, _, branch) -> + ?REV_MISSING + end, RevTree)} || #full_doc_info{rev_tree=RevTree}=Info <- Infos], NewInfos = stem_full_doc_infos(Db, NewInfos1), RemoveSeqs = @@ -900,14 +892,19 @@ copy_compact(Db, NewDb0, Retry) -> commit_data(NewDb4#db{update_seq=Db#db.update_seq}). -start_copy_compact(#db{name=Name,filepath=Filepath}=Db) -> +start_copy_compact(#db{name=Name,filepath=Filepath,header=#db_header{purge_seq=PurgeSeq}}=Db) -> CompactFile = Filepath ++ ".compact", ?LOG_DEBUG("Compaction process spawned for db \"~s\"", [Name]), case couch_file:open(CompactFile) of {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]), @@ -915,8 +912,16 @@ 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), + NewDb2 = if PurgeSeq > 0 -> + {ok, PurgedIdsRevs} = couch_db:get_last_purged(Db), + {ok, Pointer} = couch_file:append_term(Fd, PurgedIdsRevs), + NewDb#db{header=Header#db_header{purge_seq=PurgeSeq, purged_docs=Pointer}}; + true -> + NewDb + end, unlink(Fd), - NewDb2 = copy_compact(Db, NewDb, Retry), - close_db(NewDb2), + + NewDb3 = copy_compact(Db, NewDb2, Retry), + close_db(NewDb3), gen_server:cast(Db#db.main_pid, {compact_done, CompactFile}). diff --git a/apps/couch/src/couch_file.erl b/apps/couch/src/couch_file.erl index 9c06a44e..3e4f29fe 100644 --- a/apps/couch/src/couch_file.erl +++ b/apps/couch/src/couch_file.erl @@ -120,7 +120,19 @@ pread_binary(Fd, Pos) -> pread_iolist(Fd, Pos) -> - gen_server:call(Fd, {pread_iolist, Pos}, infinity). + case gen_server:call(Fd, {pread_iolist, Pos}, infinity) of + {ok, IoList, <<>>} -> + {ok, IoList}; + {ok, IoList, Md5} -> + case couch_util:md5(IoList) of + Md5 -> + {ok, IoList}; + _ -> + exit({file_corruption, <<"file corruption">>}) + end; + Error -> + Error + end. %%---------------------------------------------------------------------- %% Purpose: The length of a file, in bytes. @@ -287,15 +299,10 @@ handle_call({pread_iolist, Pos}, _From, File) -> <<1:1/integer,Len:31/integer>> -> % an MD5-prefixed term {Md5AndIoList, _} = read_raw_iolist_int(File, NextPos, Len+16), {Md5, IoList} = extract_md5(Md5AndIoList), - case couch_util:md5(IoList) of - Md5 -> - {reply, {ok, IoList}, File}; - _ -> - {stop, file_corruption, {error,file_corruption}, File} - end; + {reply, {ok, IoList, Md5}, File}; <<0:1/integer,Len:31/integer>> -> {Iolist, _} = read_raw_iolist_int(File, NextPos, Len), - {reply, {ok, Iolist}, File} + {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), diff --git a/apps/couch/src/couch_httpd.erl b/apps/couch/src/couch_httpd.erl index 2b952656..0d9abde6 100644 --- a/apps/couch/src/couch_httpd.erl +++ b/apps/couch/src/couch_httpd.erl @@ -769,7 +769,13 @@ error_headers(#httpd{mochi_req=MochiReq}=Req, Code, ErrorStr, ReasonStr) -> {Code, []}; match -> AuthRedirectBin = ?l2b(AuthRedirect), - UrlReturn = ?l2b(couch_util:url_encode(MochiReq:get(path))), + % 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 diff --git a/apps/couch/src/couch_httpd_db.erl b/apps/couch/src/couch_httpd_db.erl index cf4e2120..217a2d03 100644 --- a/apps/couch/src/couch_httpd_db.erl +++ b/apps/couch/src/couch_httpd_db.erl @@ -576,7 +576,7 @@ db_doc_req(#httpd{method='GET'}=Req, Db, DocId) -> {ok, Results} = couch_db:open_doc_revs(Db, DocId, Revs, Options), AcceptedTypes = case couch_httpd:header_value(Req, "Accept") of undefined -> []; - AcceptHeader -> string:tokens(AcceptHeader, "; ") + AcceptHeader -> string:tokens(AcceptHeader, ", ") end, case lists:member("multipart/mixed", AcceptedTypes) of false -> @@ -736,34 +736,34 @@ send_doc_efficiently(Req, #doc{atts=Atts}=Doc, Headers, Options) -> 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), + Boundary,JsonBytes, Atts, true), CType = {<<"Content-Type">>, ContentType}, {ok, Resp} = start_response_length(Req, 200, [CType|Headers], Len), couch_doc:doc_to_multi_part_stream(Boundary,JsonBytes,Atts, - fun(Data) -> couch_httpd:send(Resp, Data) end, false) + fun(Data) -> couch_httpd:send(Resp, Data) end, true) end; false -> send_json(Req, 200, Headers, couch_doc:to_json_obj(Doc, Options)) end. -send_docs_multipart(Req, Results, Options) -> +send_docs_multipart(Req, Results, Options1) -> OuterBoundary = couch_uuids:random(), InnerBoundary = couch_uuids:random(), + Options = [attachments, follows, att_encoding_info | Options1], 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])), + JsonBytes = ?JSON_ENCODE(couch_doc:to_json_obj(Doc, Options)), {ContentType, _Len} = couch_doc:len_doc_to_multi_part_stream( - InnerBoundary, JsonBytes, Atts, false), + InnerBoundary, JsonBytes, Atts, true), 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), + end, true), couch_httpd:send_chunk(Resp, <<"\r\n--", OuterBoundary/binary>>); ({{not_found, missing}, RevId}) -> RevStr = couch_doc:rev_to_str(RevId), @@ -1020,8 +1020,10 @@ db_attachment_req(#httpd{method=Method,mochi_req=MochiReq}=Req, Db, DocId, FileN end end, - #doc{atts=Atts} = Doc, + #doc{atts=Atts, revs = {Pos, Revs}} = Doc, DocEdited = Doc#doc{ + % prune revision list as a workaround for key tree bug (COUCHDB-902) + revs = {Pos, case Revs of [] -> []; [Hd|_] -> [Hd] end}, atts = NewAtt ++ [A || A <- Atts, A#att.name /= FileName] }, {ok, UpdatedRev} = couch_db:update_doc(Db, DocEdited, []), diff --git a/apps/couch/src/couch_httpd_misc_handlers.erl b/apps/couch/src/couch_httpd_misc_handlers.erl index 7b09dccd..7a149d11 100644 --- a/apps/couch/src/couch_httpd_misc_handlers.erl +++ b/apps/couch/src/couch_httpd_misc_handlers.erl @@ -93,10 +93,17 @@ handle_replicate_req(#httpd{method='POST'}=Req) -> {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"). diff --git a/apps/couch/src/couch_httpd_rewrite.erl b/apps/couch/src/couch_httpd_rewrite.erl index ca4ac1f0..6c3d0e3c 100644 --- a/apps/couch/src/couch_httpd_rewrite.erl +++ b/apps/couch/src/couch_httpd_rewrite.erl @@ -126,7 +126,10 @@ handle_rewrite_req(#httpd{ case couch_util:get_value(<<"rewrites">>, Props) of undefined -> couch_httpd:send_error(Req, 404, <<"rewrite_error">>, - <<"Invalid path.">>); + <<"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], diff --git a/apps/couch/src/couch_httpd_view.erl b/apps/couch/src/couch_httpd_view.erl index e1a0dfad..cb387d1b 100644 --- a/apps/couch/src/couch_httpd_view.erl +++ b/apps/couch/src/couch_httpd_view.erl @@ -365,6 +365,8 @@ validate_view_query(group_level, Value, Args) -> 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 -> diff --git a/apps/couch/src/couch_js_functions.hrl b/apps/couch/src/couch_js_functions.hrl index 1f314f6e..32573a90 100644 --- a/apps/couch/src/couch_js_functions.hrl +++ b/apps/couch/src/couch_js_functions.hrl @@ -31,7 +31,7 @@ throw({forbidden: 'doc.name is required'}); } - if (!(newDoc.roles && (typeof newDoc.roles.length !== 'undefined'))) { + if (newDoc.roles && !isArray(newDoc.roles)) { throw({forbidden: 'doc.roles must be an array'}); } diff --git a/apps/couch/src/couch_key_tree.erl b/apps/couch/src/couch_key_tree.erl index 4fe09bf3..6701da58 100644 --- a/apps/couch/src/couch_key_tree.erl +++ b/apps/couch/src/couch_key_tree.erl @@ -12,104 +12,107 @@ -module(couch_key_tree). --export([merge/2, find_missing/2, get_key_leafs/2, get_full_key_paths/2, get/2]). +-export([merge/3, find_missing/2, get_key_leafs/2, get_full_key_paths/2, get/2]). -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(), pos_integer()) -> {[path()], + conflicts | no_conflicts}. +merge(Paths, Path, Depth) -> + {Merged, Conflicts} = merge(Paths, Path), + {stem(Merged, Depth), Conflicts}. + +-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]), + {ok, 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 +162,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}. @@ -321,7 +324,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/apps/couch/src/couch_rep.erl b/apps/couch/src/couch_rep.erl index 126639e0..c804b49d 100644 --- a/apps/couch/src/couch_rep.erl +++ b/apps/couch/src/couch_rep.erl @@ -15,7 +15,7 @@ -export([init/1, handle_call/3, handle_cast/2, handle_info/2, terminate/2, code_change/3]). --export([replicate/2, checkpoint/1, start_link/3]). +-export([replicate/2, checkpoint/1]). -include("couch_db.hrl"). @@ -34,6 +34,7 @@ start_seq, history, + session_id, source_log, target_log, rep_starttime, @@ -46,12 +47,11 @@ committed_seq = 0, stats = nil, - doc_ids = nil + doc_ids = nil, + source_db_update_notifier = nil, + target_db_update_notifier = nil }). -start_link(Id, PostBody, UserCtx) -> - gen_server:start_link(?MODULE, [Id, PostBody, UserCtx], []). - %% convenience function to do a simple replication from the shell replicate(Source, Target) when is_list(Source) -> replicate(?l2b(Source), Target); @@ -64,7 +64,7 @@ replicate(Source, Target) when is_binary(Source), is_binary(Target) -> replicate({Props}=PostBody, UserCtx) -> {BaseId, Extension} = make_replication_id(PostBody, UserCtx), Replicator = {BaseId ++ Extension, - {?MODULE, start_link, [BaseId, PostBody, UserCtx]}, + {gen_server, start_link, [?MODULE, [BaseId, PostBody, UserCtx], []]}, temporary, 1, worker, @@ -83,15 +83,10 @@ replicate({Props}=PostBody, UserCtx) -> false -> Server = start_replication_server(Replicator), - Continuous = couch_util:get_value(<<"continuous">>, Props, false), - Async = couch_util:get_value(<<"async">>, Props, false), - case {Continuous, Async} of - {true, _} -> + case couch_util:get_value(<<"continuous">>, Props, false) of + true -> {ok, {continuous, ?l2b(BaseId)}}; - {_, true} -> - spawn(fun() -> get_result(Server, PostBody, UserCtx) end), - Server; - _ -> + false -> get_result(Server, PostBody, UserCtx) end end. @@ -113,8 +108,10 @@ get_result(Server, PostBody, UserCtx) -> end. init(InitArgs) -> - try do_init(InitArgs) - catch _:Error -> + try + do_init(InitArgs) + catch + throw:Error -> {stop, Error} end. @@ -199,12 +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 = couch_util:get_value(instance_start_time, SourceInfo), tgt_starttime = couch_util:get_value(instance_start_time, TargetInfo), - doc_ids = DocIds + doc_ids = DocIds, + source_db_update_notifier = source_db_update_notifier(Source), + target_db_update_notifier = target_db_update_notifier(Target) }, {ok, State}. @@ -212,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)}; @@ -221,16 +235,14 @@ handle_cast(_Msg, State) -> {noreply, State}. handle_info({missing_revs_checkpoint, SourceSeq}, State) -> - couch_task_status:update("MR Processed source update #~p of ~p", - [SourceSeq, seqnum(State#state.source)]), + couch_task_status:update("MR Processed source update #~p", [SourceSeq]), {noreply, schedule_checkpoint(State#state{committed_seq = SourceSeq})}; handle_info({writer_checkpoint, SourceSeq}, #state{committed_seq=N} = State) when SourceSeq > N -> MissingRevs = State#state.missing_revs, ok = gen_server:cast(MissingRevs, {update_committed_seq, SourceSeq}), - couch_task_status:update("W Processed source update #~p of ~p", - [SourceSeq, seqnum(State#state.source)]), + couch_task_status:update("W Processed source update #~p", [SourceSeq]), {noreply, schedule_checkpoint(State#state{committed_seq = SourceSeq})}; handle_info({writer_checkpoint, _}, State) -> {noreply, State}; @@ -239,14 +251,8 @@ handle_info({update_stats, Key, N}, State) -> ets:update_counter(State#state.stats, Key, N), {noreply, State}; -handle_info({'DOWN', _, _, Pid, _}, State) -> - Me = node(), - case erlang:node(Pid) of - Me -> - ?LOG_INFO("replication terminating - local DB is shutting down", []); - Node -> - ?LOG_INFO("replication terminating - DB on ~p is shutting down", [Node]) - end, +handle_info({'DOWN', _, _, _, _}, State) -> + ?LOG_INFO("replication terminating because local DB is shutting down", []), timer:cancel(State#state.checkpoint_scheduled), {stop, shutdown, State}; @@ -293,35 +299,40 @@ code_change(_OldVsn, State, _Extra) -> % internal funs start_replication_server(Replicator) -> - start_replication_server(Replicator, fun start_child/1). - -start_replication_server(Replicator, StartFun) -> - case StartFun(Replicator) of + RepId = element(1, Replicator), + case supervisor:start_child(couch_rep_sup, Replicator) of {ok, Pid} -> + ?LOG_INFO("starting new replication ~p at ~p", [RepId, Pid]), Pid; {error, already_present} -> - start_replication_server(Replicator, fun restart_child/1); + case supervisor:restart_child(couch_rep_sup, RepId) of + {ok, Pid} -> + ?LOG_INFO("starting replication ~p at ~p", [RepId, Pid]), + Pid; + {error, running} -> + %% this error occurs if multiple replicators are racing + %% each other to start and somebody else won. Just grab + %% the Pid by calling start_child again. + {error, {already_started, Pid}} = + supervisor:start_child(couch_rep_sup, Replicator), + ?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>>}); + {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, running} -> - Children = supervisor:which_children(couch_rep_sup), - {value, {_, Pid, _, _}} = lists:keysearch(Replicator, 1, Children), - Pid; - % sadly both seem to be needed. I don't know why. {error, {{db_not_found, DbUrl}, _}} -> throw({db_not_found, <<"could not open ", DbUrl/binary>>}); - {error, {db_not_found, DbUrl}} -> - throw({db_not_found, <<"could not open ", DbUrl/binary>>}); - {error, {node_not_connected, Node}} -> - throw({node_not_connected, Node}) + {error, {{unauthorized, DbUrl}, _}} -> + throw({unauthorized, + <<"unauthorized to access database ", DbUrl/binary>>}) end. -start_child(Replicator) -> - supervisor:start_child(couch_rep_sup, Replicator). - -restart_child(Replicator) -> - supervisor:restart_child(couch_rep_sup, element(1, Replicator)). - compare_replication_logs(SrcDoc, TgtDoc) -> #doc{body={RepRecProps}} = SrcDoc, #doc{body={RepRecPropsTgt}} = TgtDoc, @@ -373,15 +384,9 @@ close_db(Db) -> couch_db:close(Db). dbname(#http_db{url = Url}) -> - strip_password(Url); -dbname(#db{name = Name, main_pid = MainPid}) -> - ?l2b([Name, " (", pid_to_list(MainPid), ")"]). - -strip_password(Url) -> - re:replace(Url, - "http(s)?://([^:]+):[^@]+@(.*)$", - "http\\1://\\2:*****@\\3", - [{return, list}]). + couch_util:url_strip_password(Url); +dbname(#db{name = Name}) -> + Name. dbinfo(#http_db{} = Db) -> {DbProps} = couch_rep_httpc:request(Db), @@ -445,13 +450,20 @@ do_terminate(State) -> false -> [gen_server:reply(R, retry) || R <- OtherListeners] end, + couch_task_status:update("Finishing"), terminate_cleanup(State). -terminate_cleanup(#state{source=Source, target=Target, stats=Stats}) -> - couch_task_status:update("Finishing"), - close_db(Target), - close_db(Source), - ets:delete(Stats). +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; @@ -476,12 +488,7 @@ maybe_append_options(Options, Props) -> make_replication_id({Props}, UserCtx) -> %% funky algorithm to preserve backwards compatibility - case couch_util:get_value(<<"use_hostname">>, Props, false) of - true -> - {ok, HostName} = inet:gethostname(); - false -> - HostName = couch_config:get("replication", "hostname", "cloudant.com") - end, + {ok, HostName} = inet:gethostname(), % Port = mochiweb_socket_server:get(couch_httpd, port), Src = get_rep_endpoint(UserCtx, couch_util:get_value(<<"source">>, Props)), Tgt = get_rep_endpoint(UserCtx, couch_util:get_value(<<"target">>, Props)), @@ -504,22 +511,15 @@ make_replication_id({Props}, UserCtx) -> maybe_add_trailing_slash(Url) -> re:replace(Url, "[^/]$", "&/", [{return, list}]). -get_rep_endpoint(UserCtx, {Props}) -> - case couch_util:get_value(<<"url">>, Props) of +get_rep_endpoint(_UserCtx, {Props}) -> + 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 -> - Node = couch_util:get_value(<<"node">>, Props), - Name = couch_util:get_value(<<"name">>, Props), - {Node, Name, UserCtx}; - RawUrl -> - Url = maybe_add_trailing_slash(RawUrl), - {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} -> - {remote, Url, [{?b2l(K),?b2l(V)} || {K,V} <- BinHeaders], OAuth} - end + {remote, Url, [{?b2l(K),?b2l(V)} || {K,V} <- BinHeaders]}; + {OAuth} -> + {remote, Url, [{?b2l(K),?b2l(V)} || {K,V} <- BinHeaders], OAuth} end; get_rep_endpoint(_UserCtx, <<"http://",_/binary>>=Url) -> {remote, maybe_add_trailing_slash(Url), []}; @@ -533,43 +533,27 @@ open_replication_log(#http_db{}=Db, RepId) -> Req = Db#http_db{resource=couch_util:url_encode(DocId)}, case couch_rep_httpc:request(Req) of {[{<<"error">>, _}, {<<"reason">>, _}]} -> - % ?LOG_DEBUG("didn't find a replication log for ~s", [Db#http_db.url]), + ?LOG_DEBUG("didn't find a replication log for ~s", [Db#http_db.url]), #doc{id=?l2b(DocId)}; Doc -> - % ?LOG_DEBUG("found a replication log for ~s", [Db#http_db.url]), + ?LOG_DEBUG("found a replication log for ~s", [Db#http_db.url]), couch_doc:from_json_obj(Doc) end; open_replication_log(Db, RepId) -> DocId = ?l2b(?LOCAL_DOC_PREFIX ++ RepId), case couch_db:open_doc(Db, DocId, []) of {ok, Doc} -> - % ?LOG_DEBUG("found a replication log for ~s", [Db#db.name]), + ?LOG_DEBUG("found a replication log for ~s", [Db#db.name]), Doc; _ -> - % ?LOG_DEBUG("didn't find a replication log for ~s", [Db#db.name]), + ?LOG_DEBUG("didn't find a replication log for ~s", [Db#db.name]), #doc{id=DocId} end. open_db(Props, UserCtx, ProxyParams) -> open_db(Props, UserCtx, ProxyParams, false). -open_db(<<"http://",_/binary>>=Url, _, ProxyParams, Create) -> - open_remote_db({[{<<"url">>,Url}]}, ProxyParams, Create); -open_db(<<"https://",_/binary>>=Url, _, ProxyParams, Create) -> - open_remote_db({[{<<"url">>,Url}]}, ProxyParams, Create); -open_db({Props}, UserCtx, ProxyParams, Create) -> - case couch_util:get_value(<<"url">>, Props) of - undefined -> - Node = couch_util:get_value(<<"node">>, Props, node()), - DbName = couch_util:get_value(<<"name">>, Props), - open_local_db(Node, DbName, UserCtx, Create); - _Url -> - open_remote_db({Props}, ProxyParams, Create) - end; -open_db(<<DbName/binary>>, UserCtx, _ProxyParams, Create) -> - open_local_db(node(), DbName, UserCtx, Create). - -open_remote_db({Props}, ProxyParams, CreateTarget) -> +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, {[]}), @@ -580,33 +564,34 @@ open_remote_db({Props}, ProxyParams, CreateTarget) -> auth = AuthProps, headers = lists:ukeymerge(1, Headers, DefaultHeaders) }, - Db = Db1#http_db{options = Db1#http_db.options ++ ProxyParams}, - couch_rep_httpc:db_exists(Db, CreateTarget). - -open_local_db(Node, DbName, UserCtx, Create) when is_binary(Node) -> - try open_local_db(list_to_existing_atom(?b2l(Node)), DbName, UserCtx, Create) - catch error:badarg -> - ?LOG_ERROR("unknown replication node ~s", [Node]), - throw({node_not_connected, Node}) end; -open_local_db(Node, DbName, UserCtx, Create) when is_atom(Node) -> - case catch gen_server:call({couch_server, Node}, {open, DbName, []}, infinity) of - {ok, #db{} = Db} -> - couch_db:monitor(Db), - Db#db{fd_monitor = erlang:monitor(process, Db#db.fd)}; - {ok, MainPid} when is_pid(MainPid) -> - {ok, Db} = couch_db:open_ref_counted(MainPid, UserCtx), - couch_db:monitor(Db), - Db; - {not_found, no_db_file} when Create =:= false-> - throw({db_not_found, DbName}); - {not_found, no_db_file} -> - ok = couch_httpd:verify_is_server_admin(UserCtx), - couch_server:create(DbName, [{user_ctx, UserCtx}]); - {'EXIT', {{nodedown, Node}, _Stack}} -> - throw({node_not_connected, couch_util:to_binary(Node)}); - {'EXIT', {noproc, {gen_server,call,_}}} -> - timer:sleep(1000), - throw({noproc, couch_server, Node}) + 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, _, 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) -> @@ -628,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, @@ -637,14 +623,8 @@ do_checkpoint(State) -> } = State, case commit_to_both(Source, Target, NewSeqNum) of {SrcInstanceStartTime, TgtInstanceStartTime} -> - ?LOG_DEBUG("recording a checkpoint for ~s -> ~s at source update_seq ~p" - " of ~p", [dbname(Source), dbname(Target), NewSeqNum, seqnum(Source)]), - SessionId = couch_uuids:new(), - TargetNode = case Target of #db{main_pid=MainPid} -> - erlang:node(MainPid); - _ -> - http - end, + ?LOG_INFO("recording a checkpoint for ~s -> ~s at source update_seq ~p", + [dbname(Source), dbname(Target), NewSeqNum]), NewHistoryEntry = {[ {<<"session_id">>, SessionId}, {<<"start_time">>, list_to_binary(ReplicationStartTime)}, @@ -663,7 +643,6 @@ do_checkpoint(State) -> NewRepHistory = {[ {<<"session_id">>, SessionId}, {<<"source_last_seq">>, NewSeqNum}, - {<<"target_node">>, TargetNode}, {<<"history">>, lists:sublist([NewHistoryEntry | OldHistory], 50)} ]}, @@ -683,9 +662,7 @@ do_checkpoint(State) -> "yourself?)", []), State end; - Else -> - ?LOG_INFO("wanted ~p, got ~p from commit_to_both", [ - {SrcInstanceStartTime, TgtInstanceStartTime}, Else]), + _Else -> ?LOG_INFO("rebooting ~s -> ~s from last known replication checkpoint", [dbname(Source), dbname(Target)]), #state{ @@ -717,31 +694,30 @@ commit_to_both(Source, Target, RequiredSeq) -> {SrcCommitPid, Timestamp} -> Timestamp; {'EXIT', SrcCommitPid, {http_request_failed, _}} -> - nil; - {'EXIT', SrcCommitPid, {noproc, {gen_server, call, [_]}}} -> - nil; % DB crashed, this should trigger a reboot - {'EXIT', SrcCommitPid, Else} -> - ?LOG_ERROR("new error code for crashed replication commit ~p", [Else]), - nil + exit(replication_link_failure) end, {SourceStartTime, TargetStartTime}. 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, - headers = couch_util:proplist_apply_field({"Content-Type", "application/json"}, Headers) + headers = Headers1 }, {ResultProps} = couch_rep_httpc:request(Req), true = couch_util:get_value(<<"ok">>, ResultProps), couch_util:get_value(<<"instance_start_time">>, ResultProps); -ensure_full_commit(#db{name=DbName, main_pid=Pid} = Target) -> - TargetNode = erlang:node(Pid), - {ok, NewDb} = rpc:call(TargetNode, couch_db, open_int, [DbName, []]), +ensure_full_commit(Target) -> + {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, - catch couch_db:close(NewDb), + couch_db:close(NewDb), if UpdateSeq > CommitSeq -> ?LOG_DEBUG("target needs a full commit: update ~p commit ~p", [UpdateSeq, CommitSeq]), @@ -753,11 +729,16 @@ ensure_full_commit(#db{name=DbName, main_pid=Pid} = Target) -> end. 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, qs = [{seq, RequiredSeq}], - headers = couch_util:proplist_apply_field({"Content-Type", "application/json"}, Headers) + headers = Headers1 }, {ResultProps} = couch_rep_httpc:request(Req), case couch_util:get_value(<<"ok">>, ResultProps) of @@ -801,11 +782,6 @@ up_to_date(Source, Seq) -> couch_db:close(NewDb), T. -seqnum(#http_db{}) -> - -1; -seqnum(Db) -> - Db#db.update_seq. - parse_proxy_params(ProxyUrl) when is_binary(ProxyUrl) -> parse_proxy_params(?b2l(ProxyUrl)); parse_proxy_params([]) -> @@ -820,3 +796,27 @@ parse_proxy_params(ProxyUrl) -> 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/apps/couch/src/couch_rep_att.erl b/apps/couch/src/couch_rep_att.erl index 476c64d4..72c723e8 100644 --- a/apps/couch/src/couch_rep_att.erl +++ b/apps/couch/src/couch_rep_att.erl @@ -105,7 +105,7 @@ 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/apps/couch/src/couch_rep_changes_feed.erl b/apps/couch/src/couch_rep_changes_feed.erl index 66696912..032f62a3 100644 --- a/apps/couch/src/couch_rep_changes_feed.erl +++ b/apps/couch/src/couch_rep_changes_feed.erl @@ -43,9 +43,10 @@ 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 couch_util:get_value(<<"continuous">>, PostProps, false) of false -> @@ -83,27 +84,32 @@ init([_Parent, #http_db{}=Source, Since, PostProps] = Args) -> resource = "_changes", qs = QS, conn = Pid, - options = [{stream_to, {self(), once}}, {response_format, binary}], + 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), @@ -181,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); @@ -198,16 +204,27 @@ handle_info({ibrowse_async_response_end, Id}, #state{reqid=Id} = State) -> handle_info({'EXIT', From, normal}, #state{changes_loop=From} = State) -> handle_feed_completion(State); +handle_info({'EXIT', From, normal}, #state{conn=From, complete=true} = State) -> + {noreply, State}; + 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({'EXIT', _From, normal}, State) -> - {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) -> - ?LOG_DEBUG("unexpected message at changes_feed ~p", [Msg]), - {noreply, 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{ @@ -215,8 +232,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}. @@ -257,12 +273,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]), @@ -367,20 +388,15 @@ 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:exit_after(31000, changes_timeout), + {ok, Timeout} = timer:send_after(31000, changes_timeout), put(timeout, Timeout), ibrowse:stream_next(S#state.reqid); maybe_stream_next(_) -> timer:cancel(get(timeout)). -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}. +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/apps/couch/src/couch_rep_httpc.erl b/apps/couch/src/couch_rep_httpc.erl index 3b11b869..8153fdcf 100644 --- a/apps/couch/src/couch_rep_httpc.erl +++ b/apps/couch/src/couch_rep_httpc.erl @@ -15,7 +15,8 @@ -include_lib("ibrowse/include/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). @@ -72,6 +73,7 @@ db_exists(Req, CanonicalUrl, CreateDB) -> #http_db{ auth = Auth, headers = Headers0, + options = Options, url = Url } = Req, HeadersFun = fun(Method) -> @@ -84,11 +86,13 @@ 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", _, _} -> + config_http(CanonicalUrl), Req#http_db{url = CanonicalUrl}; {ok, "301", RespHeaders, _} -> RedirectUrl = redirect_url(RespHeaders, Req#http_db.url), @@ -96,11 +100,26 @@ db_exists(Req, CanonicalUrl, CreateDB) -> {ok, "302", RespHeaders, _} -> 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. +config_http(Url) -> + #url{host = Host, port = Port} = ibrowse_lib:parse_url(Url), + ok = ibrowse:set_max_sessions(Host, Port, list_to_integer( + couch_config:get("replicator", "max_http_sessions", "20"))), + ok = ibrowse:set_max_pipeline_size(Host, Port, list_to_integer( + couch_config:get("replicator", "max_http_pipeline_size", "50"))), + ok = couch_config:register( + fun("replicator", "max_http_sessions", MaxSessions) -> + ibrowse:set_max_sessions(Host, Port, list_to_integer(MaxSessions)); + ("replicator", "max_http_pipeline_size", PipeSize) -> + ibrowse:set_max_pipeline_size(Host, Port, list_to_integer(PipeSize)) + end). + redirect_url(RespHeaders, OrigUrl) -> MochiHeaders = mochiweb_headers:make(RespHeaders), RedUrl = mochiweb_headers:get_value("Location", MochiHeaders), @@ -167,7 +186,7 @@ process_response({error, Reason}, Req) -> pause = Pause } = Req, ShortReason = case Reason of - connection_closed -> + sel_conn_closed -> connection_closed; {'EXIT', {noproc, _}} -> noproc; @@ -203,8 +222,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) -> @@ -243,3 +261,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/apps/couch/src/couch_rep_missing_revs.erl b/apps/couch/src/couch_rep_missing_revs.erl index 1eff6774..9809ca5e 100644 --- a/apps/couch/src/couch_rep_missing_revs.erl +++ b/apps/couch/src/couch_rep_missing_revs.erl @@ -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,15 +132,16 @@ 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({Props}) -> diff --git a/apps/couch/src/couch_rep_reader.erl b/apps/couch/src/couch_rep_reader.erl index 46633994..a7ae45a8 100644 --- a/apps/couch/src/couch_rep_reader.erl +++ b/apps/couch/src/couch_rep_reader.erl @@ -20,12 +20,9 @@ -import(couch_util, [url_encode/1]). -define (BUFFER_SIZE, 1000). --define (MAX_CONCURRENT_REQUESTS, 10). --define (MAX_CONNECTIONS, 20). --define (MAX_PIPELINE_SIZE, 50). +-define (MAX_CONCURRENT_REQUESTS, 100). -include("couch_db.hrl"). --include_lib("ibrowse/include/ibrowse.hrl"). -record (state, { parent, @@ -53,14 +50,9 @@ next(Pid) -> 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), - ibrowse:set_max_sessions(Host, Port, ?MAX_CONNECTIONS), - ibrowse:set_max_pipeline_size(Host, Port, ?MAX_PIPELINE_SIZE); - true -> ok end, Self = self(), ReaderLoop = spawn_link( - fun() -> reader_loop(Self, Source, MissingRevs_or_DocIds) end + fun() -> reader_loop(Self, Parent, Source, MissingRevs_or_DocIds) end ), MissingRevs = case MissingRevs_or_DocIds of Pid when is_pid(Pid) -> @@ -230,7 +222,7 @@ 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}, {att_encoding_info,true}], @@ -246,36 +238,48 @@ 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{} = DbS, DocId) -> +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}] }, - case couch_rep_httpc:request(Req) of - {[{<<"error">>,<<"not_found">>}, {<<"reason">>,<<"missing">>}]} -> - []; - Json -> + {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, DocIds) when is_list(DocIds) -> - case Source of +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} -> @@ -288,7 +292,7 @@ reader_loop(ReaderServer, Source, DocIds) when is_list(DocIds) -> end, exit(complete); -reader_loop(ReaderServer, Source, MissingRevsServer) -> +reader_loop(ReaderServer, Parent, Source, MissingRevsServer) -> case couch_rep_missing_revs:next(MissingRevsServer) of complete -> exit(complete); @@ -301,22 +305,23 @@ 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. diff --git a/apps/couch/src/couch_rep_writer.erl b/apps/couch/src/couch_rep_writer.erl index dd6396fd..cf98ccfb 100644 --- a/apps/couch/src/couch_rep_writer.erl +++ b/apps/couch/src/couch_rep_writer.erl @@ -16,10 +16,10 @@ -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; @@ -28,6 +28,7 @@ writer_loop(Parent, Reader, Target) -> 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}; @@ -48,7 +49,7 @@ writer_loop(Parent, Reader, Target) -> end, couch_rep_att:cleanup(), couch_util:should_flush(), - writer_loop(Parent, Reader, Target) + writer_loop(Parent, Reader) end. write_docs(#http_db{} = Db, Docs) -> diff --git a/apps/couch/src/couch_util.erl b/apps/couch/src/couch_util.erl index 3a6e92c5..adcb4450 100644 --- a/apps/couch/src/couch_util.erl +++ b/apps/couch/src/couch_util.erl @@ -27,6 +27,7 @@ -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"). @@ -417,8 +418,8 @@ compressible_att_type(MimeType) -> ), lists:any( fun(TypeExp) -> - Regexp = "^\\s*" ++ - re:replace(TypeExp, "\\*", ".*", [{return, list}]) ++ "\\s*$", + Regexp = ["^\\s*", re:replace(TypeExp, "\\*", ".*"), + "(?:\\s*;.*?)?\\s*", $$], case re:run(MimeType, Regexp, [caseless]) of {match, _} -> true; @@ -452,3 +453,9 @@ reorder_results(Keys, SortedResults) when length(Keys) < 100 -> 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/apps/couch/src/couch_view.erl b/apps/couch/src/couch_view.erl index 9de86b82..8dca17da 100644 --- a/apps/couch/src/couch_view.erl +++ b/apps/couch/src/couch_view.erl @@ -30,11 +30,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 -> @@ -42,10 +40,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 {ok, Group} -> - 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 -> diff --git a/apps/couch/src/couch_view_compactor.erl b/apps/couch/src/couch_view_compactor.erl index f56325a4..43db9036 100644 --- a/apps/couch/src/couch_view_compactor.erl +++ b/apps/couch/src/couch_view_compactor.erl @@ -53,18 +53,22 @@ compact_group(Group, EmptyGroup) -> TaskName = <<DbName/binary, ShortName/binary>>, couch_task_status:add_task(<<"View Group Compaction">>, TaskName, <<"">>), - Fun = fun(KV, {Bt, Acc, TotalCopied}) -> + Fun = fun({DocId, _ViewIdKeys} = KV, {Bt, Acc, TotalCopied, LastId}) -> + if DocId =:= LastId -> % COUCHDB-999 + Msg = "Duplicates of ~s detected in ~s ~s - rebuild required", + exit(io_lib:format(Msg, [DocId, DbName, GroupId])); + true -> ok end, if TotalCopied rem 10000 =:= 0 -> couch_task_status:update("Copied ~p of ~p Ids (~p%)", [TotalCopied, Count, (TotalCopied*100) div Count]), {ok, Bt2} = couch_btree:add(Bt, lists:reverse([KV|Acc])), - {ok, {Bt2, [], TotalCopied+1}}; + {ok, {Bt2, [], TotalCopied+1, DocId}}; true -> - {ok, {Bt, [KV|Acc], TotalCopied+1}} + {ok, {Bt, [KV|Acc], TotalCopied+1, DocId}} end end, - {ok, _, {Bt3, Uncopied, _Total}} = couch_btree:foldl(IdBtree, Fun, - {EmptyIdBtree, [], 0}), + {ok, _, {Bt3, Uncopied, _Total, _LastId}} = couch_btree:foldl(IdBtree, Fun, + {EmptyIdBtree, [], 0, nil}), {ok, NewIdBtree} = couch_btree:add(Bt3, lists:reverse(Uncopied)), NewViews = lists:map(fun({View, EmptyView}) -> diff --git a/apps/couch/src/couch_view_group.erl b/apps/couch/src/couch_view_group.erl index 377e7516..730db185 100644 --- a/apps/couch/src/couch_view_group.erl +++ b/apps/couch/src/couch_view_group.erl @@ -74,7 +74,7 @@ start_link(InitArgs) -> end. % init creates a closure which spawns the appropriate view_updater. -init({{_, DbName, _}=InitArgs, ReturnPid, Ref}) -> +init({{_, DbName, _} = InitArgs, ReturnPid, Ref}) -> process_flag(trap_exit, true), case prepare_group(InitArgs, false) of {ok, #group{fd=Fd, current_seq=Seq}=Group} -> @@ -86,12 +86,9 @@ init({{_, DbName, _}=InitArgs, ReturnPid, Ref}) -> ignore; _ -> try couch_db:monitor(Db) after couch_db:close(Db) end, - Owner = self(), - Pid = spawn_link(fun()-> couch_view_updater:update(Owner, Group) end), {ok, #group_state{ db_name= DbName, init_args=InitArgs, - updater_pid = Pid, group=Group#group{dbname=DbName}, ref_counter=erlang:monitor(process,Fd)}} end; @@ -178,6 +175,7 @@ handle_cast({compact_done, #group{fd=NewFd, current_seq=NewSeq} = NewGroup}, group = #group{name=GroupId, fd=OldFd, sig=GroupSig}, init_args = {RootDir, DbName, _}, updater_pid = UpdaterPid, + compactor_pid = CompactorPid, ref_counter = RefCounter } = State, @@ -199,6 +197,8 @@ handle_cast({compact_done, #group{fd=NewFd, current_seq=NewSeq} = NewGroup}, end, %% cleanup old group + unlink(CompactorPid), + receive {'EXIT', CompactorPid, normal} -> ok after 0 -> ok end, unlink(OldFd), erlang:demonitor(RefCounter), @@ -426,8 +426,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">>, views=[View], + couch_db:close(Db), + {ok, set_view_sig(#group{name = <<"_temp">>, views=[View], def_lang=Language, design_options=DesignOptions})}; Error -> Error diff --git a/apps/couch/test/etap/060-kt-merging.t b/apps/couch/test/etap/060-kt-merging.t index d6b13d6d..0e481a52 100755 --- a/apps/couch/test/etap/060-kt-merging.t +++ b/apps/couch/test/etap/060-kt-merging.t @@ -15,7 +15,7 @@ main(_) -> test_util:init_code_path(), - etap:plan(16), + etap:plan(12), case (catch test()) of ok -> etap:end_tests(); @@ -26,114 +26,89 @@ main(_) -> ok. test() -> - EmptyTree = [], - One = [{0, {"1","foo",[]}}], + One = {0, {"1","foo",[]}}, TwoSibs = [{0, {"1","foo",[]}}, {0, {"2","foo",[]}}], - OneChild = [{0, {"1","foo",[{"1a", "bar", []}]}}], - TwoChild = [{0, {"1","foo", [{"1a", "bar", [{"1aa", "bar", []}]}]}}], - TwoChildSibs = [{0, {"1","foo", [{"1a", "bar", []}, - {"1b", "bar", []}]}}], - TwoChildSibs2 = [{0, {"1","foo", [{"1a", "bar", []}, - {"1b", "bar", [{"1bb", "boo", []}]}]}}], - Stemmed1b = [{1, {"1a", "bar", []}}], - Stemmed1a = [{1, {"1a", "bar", [{"1aa", "bar", []}]}}], - Stemmed1aa = [{2, {"1aa", "bar", []}}], - Stemmed1bb = [{2, {"1bb", "boo", []}}], - - etap:is( - {EmptyTree, no_conflicts}, - couch_key_tree:merge(EmptyTree, EmptyTree), - "Merging two empty trees yields an empty tree." - ), - - etap:is( - {One, no_conflicts}, - couch_key_tree:merge(EmptyTree, One), + OneChild = {0, {"1","foo",[{"1a", "bar", []}]}}, + TwoChild = {0, {"1","foo", [{"1a", "bar", [{"1aa", "bar", []}]}]}}, + TwoChildSibs = {0, {"1","foo", [{"1a", "bar", []}, + {"1b", "bar", []}]}}, + TwoChildSibs2 = {0, {"1","foo", [{"1a", "bar", []}, + {"1b", "bar", [{"1bb", "boo", []}]}]}}, + Stemmed1b = {1, {"1a", "bar", []}}, + Stemmed1a = {1, {"1a", "bar", [{"1aa", "bar", []}]}}, + Stemmed1aa = {2, {"1aa", "bar", []}}, + Stemmed1bb = {2, {"1bb", "boo", []}}, + + etap:is( + {[One], no_conflicts}, + couch_key_tree:merge([], One, 10), "The empty tree is the identity for merge." ), etap:is( - {One, no_conflicts}, - couch_key_tree:merge(One, EmptyTree), - "Merging is commutative." - ), - - etap:is( {TwoSibs, no_conflicts}, - couch_key_tree:merge(One, TwoSibs), + couch_key_tree:merge(TwoSibs, One, 10), "Merging a prefix of a tree with the tree yields the tree." ), etap:is( - {One, no_conflicts}, - couch_key_tree:merge(One, One), + {[One], no_conflicts}, + couch_key_tree:merge([One], One, 10), "Merging is reflexive." ), etap:is( - {TwoChild, no_conflicts}, - couch_key_tree:merge(TwoChild, TwoChild), + {[TwoChild], no_conflicts}, + couch_key_tree:merge([TwoChild], TwoChild, 10), "Merging two children is still reflexive." ), etap:is( - {TwoChildSibs, no_conflicts}, - couch_key_tree:merge(TwoChildSibs, TwoChildSibs), + {[TwoChildSibs], no_conflicts}, + couch_key_tree:merge([TwoChildSibs], TwoChildSibs, 10), "Merging a tree to itself is itself."), etap:is( - {TwoChildSibs, no_conflicts}, - couch_key_tree:merge(TwoChildSibs, Stemmed1b), + {[TwoChildSibs], no_conflicts}, + couch_key_tree:merge([TwoChildSibs], Stemmed1b, 10), "Merging a tree with a stem." ), etap:is( - {TwoChildSibs, no_conflicts}, - couch_key_tree:merge(Stemmed1b, TwoChildSibs), - "Merging in the opposite direction." - ), - - etap:is( - {TwoChildSibs2, no_conflicts}, - couch_key_tree:merge(TwoChildSibs2, Stemmed1bb), + {[TwoChildSibs2], no_conflicts}, + couch_key_tree:merge([TwoChildSibs2], Stemmed1bb, 10), "Merging a stem at a deeper level." ), etap:is( - {TwoChildSibs2, no_conflicts}, - couch_key_tree:merge(Stemmed1bb, TwoChildSibs2), - "Merging a deeper level in opposite order." - ), - - etap:is( - {TwoChild, no_conflicts}, - couch_key_tree:merge(TwoChild, Stemmed1aa), + {[TwoChild], no_conflicts}, + couch_key_tree:merge([TwoChild], Stemmed1aa, 10), "Merging a single tree with a deeper stem." ), etap:is( - {TwoChild, no_conflicts}, - couch_key_tree:merge(TwoChild, Stemmed1a), + {[TwoChild], no_conflicts}, + couch_key_tree:merge([TwoChild], Stemmed1a, 10), "Merging a larger stem." ), etap:is( - {Stemmed1a, no_conflicts}, - couch_key_tree:merge(Stemmed1a, Stemmed1aa), + {[Stemmed1a], no_conflicts}, + couch_key_tree:merge([Stemmed1a], Stemmed1aa, 10), "More merging." ), - Expect1 = OneChild ++ Stemmed1aa, + Expect1 = [OneChild, Stemmed1aa], etap:is( {Expect1, conflicts}, - couch_key_tree:merge(OneChild, Stemmed1aa), + couch_key_tree:merge([OneChild], Stemmed1aa, 10), "Merging should create conflicts." ), etap:is( - {TwoChild, no_conflicts}, - couch_key_tree:merge(Expect1, TwoChild), + {[TwoChild], no_conflicts}, + couch_key_tree:merge(Expect1, TwoChild, 10), "Merge should have no conflicts." ), diff --git a/couchjs/js/loop.js b/couchjs/js/loop.js index 300151e9..a988684f 100644 --- a/couchjs/js/loop.js +++ b/couchjs/js/loop.js @@ -26,6 +26,7 @@ function init_sandbox() { sandbox.start = Render.start; sandbox.send = Render.send; sandbox.getRow = Render.getRow; + sandbox.isArray = isArray; } catch (e) { log(e.toSource()); } diff --git a/couchjs/js/util.js b/couchjs/js/util.js index 9cc464c3..b55480b9 100644 --- a/couchjs/js/util.js +++ b/couchjs/js/util.js @@ -10,36 +10,50 @@ // License for the specific language governing permissions and limitations under // the License. -var resolveModule = function(names, parent, current, path) { +var resolveModule = function(names, mod, root) { if (names.length == 0) { - if (typeof current != "string") { + if (typeof mod.current != "string") { throw ["error","invalid_require_path", - 'Must require a JavaScript string, not: '+(typeof current)]; + 'Must require a JavaScript string, not: '+(typeof mod.current)]; + } + return { + current : mod.current, + parent : mod.parent, + id : mod.id, + exports : {} } - return [current, parent, path]; } // we need to traverse the path var n = names.shift(); if (n == '..') { - if (!(parent && parent.parent)) { - throw ["error", "invalid_require_path", 'Object has no parent '+JSON.stringify(current)]; + if (!(mod.parent && mod.parent.parent)) { + throw ["error", "invalid_require_path", 'Object has no parent '+JSON.stringify(mod.current)]; } - path = path.slice(0, path.lastIndexOf('/')); - return resolveModule(names, parent.parent.parent, parent.parent, path); + return resolveModule(names, { + id : mod.id.slice(0, mod.id.lastIndexOf('/')), + parent : mod.parent.parent.parent, + current : mod.parent.parent.current + }); } else if (n == '.') { - if (!parent) { - throw ["error", "invalid_require_path", 'Object has no parent '+JSON.stringify(current)]; + if (!mod.parent) { + throw ["error", "invalid_require_path", 'Object has no parent '+JSON.stringify(mod.current)]; } - return resolveModule(names, parent.parent, parent, path); + return resolveModule(names, { + parent : mod.parent.parent, + current : mod.parent.current, + id : mod.id + }); + } else if (root) { + mod = {current : root}; } - if (!current[n]) { - throw ["error", "invalid_require_path", 'Object has no property "'+n+'". '+JSON.stringify(current)]; + if (!mod.current[n]) { + throw ["error", "invalid_require_path", 'Object has no property "'+n+'". '+JSON.stringify(mod.current)]; } - var p = current; - current = current[n]; - current.parent = p; - path = path ? path + '/' + n : n; - return resolveModule(names, p, current, path); + return resolveModule(names, { + current : mod.current[n], + parent : mod, + id : mod.id ? mod.id + '/' + n : n + }); }; var Couch = { @@ -52,19 +66,17 @@ var Couch = { try { if (sandbox) { if (ddoc) { - var require = function(name, parent) { - if (!parent) {parent = {}}; - var resolved = resolveModule(name.split('/'), parent.actual, ddoc, parent.id); - var s = "function (module, exports, require) { " + resolved[0] + " }"; - var module = {id:resolved[2], actual:resolved[1]}; - module.exports = {}; + var require = function(name, module) { + module = module || {}; + var newModule = resolveModule(name.split('/'), module, ddoc); + var s = "function (module, exports, require) { " + newModule.current + " }"; try { var func = sandbox ? evalcx(s, sandbox) : eval(s); - func.apply(sandbox, [module, module.exports, function(name) {return require(name, module)}]); + func.apply(sandbox, [newModule, newModule.exports, function(name) {return require(name, newModule)}]); } catch(e) { throw ["error","compilation_error","Module require('"+name+"') raised error "+e.toSource()]; } - return module.exports; + return newModule.exports; } sandbox.require = require; } @@ -84,10 +96,15 @@ var Couch = { }, recursivelySeal : function(obj) { // seal() is broken in current Spidermonkey - seal(obj); + try { + seal(obj); + } catch (x) { + // Sealing of arrays broken in some SpiderMonkey versions. + // https://bugzilla.mozilla.org/show_bug.cgi?id=449657 + } for (var propname in obj) { - if (typeof doc[propname] == "object") { - recursivelySeal(doc[propname]); + if (typeof obj[propname] == "object") { + arguments.callee(obj[propname]); } } } @@ -105,8 +122,14 @@ function respond(obj) { function log(message) { // idea: query_server_config option for log level - if (typeof message != "string") { + if (typeof message == "xml") { + message = message.toXMLString(); + } else if (typeof message != "string") { message = Couch.toJSON(message); } - respond(["log", message]); + respond(["log", String(message)]); }; + +function isArray(obj) { + return toString.call(obj) === "[object Array]"; +} diff --git a/couchjs/js/views.js b/couchjs/js/views.js index ffe63377..2a15ee56 100644 --- a/couchjs/js/views.js +++ b/couchjs/js/views.js @@ -105,19 +105,8 @@ var Views = (function() { // ] // - /* - Immutable document support temporarily removed. + Couch.recursivelySeal(doc); - Removed because the seal function no longer works on JS 1.8 arrays, - instead returning an error. The sealing is meant to prevent map - functions from modifying the same document that is passed to other map - functions. However, only map functions in the same design document are - run together, so we have a reasonable expectation they can trust each - other. Any map fun that can't be trusted can be placed in its own - design document, and it cannot affect other map functions. - - recursivelySeal(doc); // seal to prevent map functions from changing doc - */ var buf = []; for (var i = 0; i < State.funs.length; i++) { map_results = []; diff --git a/rebar.config b/rebar.config index bcbc117b..b621564a 100644 --- a/rebar.config +++ b/rebar.config @@ -16,9 +16,9 @@ {oauth, ".*", {git, "git://github.com/cloudant/erlang-oauth.git", {tag, "CouchDB-1.0.1-rebar"}}}, {ibrowse, ".*", {git, "git://github.com/cloudant/ibrowse.git", {branch, + "couch2"}}}, + {mochiweb, ".*", {git, "git://github.com/cloudant/mochiweb.git", {branch, "couch"}}}, - {mochiweb, ".*", {git, "git://github.com/cloudant/mochiweb.git", {tag, - "CouchDB-1.0.1-rebar"}}}, {rexi, ".*", {git, "git://github.com/cloudant/rexi.git", "master"}}, {fabric, ".*", {git, "git://github.com/cloudant/fabric.git", "master"}}, {mem3, ".*", {git, "git://github.com/cloudant/mem3.git", "master"}}, diff --git a/rel/overlay/etc/default.ini b/rel/overlay/etc/default.ini index ecf340d3..cf06a1ec 100644 --- a/rel/overlay/etc/default.ini +++ b/rel/overlay/etc/default.ini @@ -19,21 +19,22 @@ docroot = {{prefix}}/share/www [httpd] port = {{backend_port}} -bind_address = 0.0.0.0 -authentication_handlers = {couch_httpd_auth, cookie_authentication_handler}, {couch_httpd_auth, default_authentication_handler} +bind_address = 127.0.0.1 +max_connections = 2048 +authentication_handlers = {couch_httpd_oauth, oauth_authentication_handler}, {couch_httpd_auth, cookie_authentication_handler}, {couch_httpd_auth, default_authentication_handler} default_handler = {couch_httpd_db, handle_request} -WWW-Authenticate = Basic realm="Cloudant Private Database" -backlog = 512 secure_rewrites = true +vhost_global_handlers = _utils, _uuids, _session, _oauth, _users +allow_jsonp = false [log] file = {{prefix}}/var/log/bigcouch.log level = info +include_sasl = true [couch_httpd_auth] +authentication_db = _users authentication_redirect = /_utils/session.html -authentication_db = users -secret = replace this with a real secret in your local.ini file require_valid_user = false timeout = 600 ; number of seconds before automatic logout auth_cache_size = 50 ; size is number of cache entries @@ -69,7 +70,6 @@ _stats = {couch_httpd_stats_handlers, handle_stats_req} _log = {couch_httpd_misc_handlers, handle_log_req} _session = {couch_httpd_auth, handle_session_req} _oauth = {couch_httpd_oauth, handle_oauth_req} -_user = {couch_httpd_auth, handle_user_req} _system = {chttpd_misc, handle_system_req} [httpd_db_handlers] @@ -110,5 +110,11 @@ compression_level = 8 ; from 1 (lowest, fastest) to 9 (highest, slowest), 0 to d compressible_types = text/*, application/javascript, application/json, application/xml [replicator] -max_http_sessions = 10 -max_http_pipeline_size = 10 +max_http_sessions = 20 +max_http_pipeline_size = 50 +; set to true to validate peer certificates +verify_ssl_certificates = false +; file containing a list of peer trusted certificates (PEM format) +; ssl_trusted_certificates_file = /etc/ssl/certs/ca-certificates.crt +; maximum peer certificate depth (must be set even if certificate validation is off) +ssl_certificate_max_depth = 3 diff --git a/rel/overlay/share/www/database.html b/rel/overlay/share/www/database.html index 39507398..9a9f121e 100644 --- a/rel/overlay/share/www/database.html +++ b/rel/overlay/share/www/database.html @@ -37,9 +37,13 @@ specific language governing permissions and limitations under the License. $(function() { if (page.redirecting) return; - $("h1 strong").html('<a href="?' + page.db.name + '">' + page.db.name + '</a>'); + $("h1 strong").text(page.db.name); var viewPath = page.viewName || "_all_docs"; if (viewPath != "_temp_view" && viewPath != "_design_docs") { + viewPath = $.map(viewPath.split("/"), function (part) { + return encodeURIComponent(part); + }).join("/"); + $("h1 a.raw").attr("href", "/" + encodeURIComponent(page.db.name) + "/" + viewPath); } @@ -71,17 +75,17 @@ specific language governing permissions and limitations under the License. }); // Restore preferences/state - $("#documents thead th.key").toggleClass("desc", $.futon.storage.get("desc")); - var reduce = $.futon.storage.get("reduce"); + $("#documents thead th.key").toggleClass("desc", !!$.futon.storage.get("desc")); + var reduce = !!$.futon.storage.get("reduce"); $("#reduce :checkbox")[0].checked = reduce; - $("#grouplevel select").val($.futon.storage.get("group_level")); + $("#grouplevel select").val(parseInt($.futon.storage.get("group_level"))); $("#grouplevel").toggleClass("disabled", !reduce).find("select").each(function() { this.disabled = !reduce; }); - $("#perpage").val($.futon.storage.get("per_page")); + $("#perpage").val(parseInt($.futon.storage.get("per_page"))); - var staleViews = $.futon.storage.get("stale"); + var staleViews = !!$.futon.storage.get("stale"); $("#staleviews :checkbox")[0].checked = staleViews; page.populateViewsMenu(); diff --git a/rel/overlay/share/www/image/spinner.gif b/rel/overlay/share/www/image/spinner.gif Binary files differindex f27d7cd4..6239655e 100644 --- a/rel/overlay/share/www/image/spinner.gif +++ b/rel/overlay/share/www/image/spinner.gif diff --git a/rel/overlay/share/www/index.html b/rel/overlay/share/www/index.html index e34dbde1..975f5986 100644 --- a/rel/overlay/share/www/index.html +++ b/rel/overlay/share/www/index.html @@ -34,7 +34,7 @@ specific language governing permissions and limitations under the License. this.updateSelection(location.pathname + "index.html"); }); } - var dbsPerPage = $.futon.storage.get("per_page"); + var dbsPerPage = parseInt($.futon.storage.get("per_page")); if (dbsPerPage) $("#perpage").val(dbsPerPage); $("#perpage").change(function() { page.updateDatabaseListing(); diff --git a/rel/overlay/share/www/script/couch.js b/rel/overlay/share/www/script/couch.js index 33fd82ba..ca860bd5 100644 --- a/rel/overlay/share/www/script/couch.js +++ b/rel/overlay/share/www/script/couch.js @@ -22,17 +22,17 @@ function CouchDB(name, httpHeaders) { this.last_req = null; this.request = function(method, uri, requestOptions) { - requestOptions = requestOptions || {} - requestOptions.headers = combine(requestOptions.headers, httpHeaders) + requestOptions = requestOptions || {}; + requestOptions.headers = combine(requestOptions.headers, httpHeaders); return CouchDB.request(method, uri, requestOptions); - } + }; // Creates the database on the server this.createDb = function() { this.last_req = this.request("PUT", this.uri); CouchDB.maybeThrowError(this.last_req); return JSON.parse(this.last_req.responseText); - } + }; // Deletes the database on the server this.deleteDb = function() { @@ -42,7 +42,7 @@ function CouchDB(name, httpHeaders) { } CouchDB.maybeThrowError(this.last_req); return JSON.parse(this.last_req.responseText); - } + }; // Save a document to the database this.save = function(doc, options) { @@ -57,7 +57,7 @@ function CouchDB(name, httpHeaders) { var result = JSON.parse(this.last_req.responseText); doc._rev = result.rev; return result; - } + }; // Open a document from the database this.open = function(docId, options) { @@ -68,7 +68,7 @@ function CouchDB(name, httpHeaders) { } CouchDB.maybeThrowError(this.last_req); return JSON.parse(this.last_req.responseText); - } + }; // Deletes a document from the database this.deleteDoc = function(doc) { @@ -79,7 +79,7 @@ function CouchDB(name, httpHeaders) { doc._rev = result.rev; //record rev in input document doc._deleted = true; return result; - } + }; // Deletes an attachment from a document this.deleteDocAttachment = function(doc, attachment_name) { @@ -89,18 +89,18 @@ function CouchDB(name, httpHeaders) { var result = JSON.parse(this.last_req.responseText); doc._rev = result.rev; //record rev in input document return result; - } + }; this.bulkSave = function(docs, options) { // first prepoulate the UUIDs for new documents - var newCount = 0 + var newCount = 0; for (var i=0; i<docs.length; i++) { if (docs[i]._id == undefined) { newCount++; } } var newUuids = CouchDB.newUuids(docs.length); - var newCount = 0 + var newCount = 0; for (var i=0; i<docs.length; i++) { if (docs[i]._id == undefined) { docs[i]._id = newUuids.pop(); @@ -127,13 +127,13 @@ function CouchDB(name, httpHeaders) { } return results; } - } + }; this.ensureFullCommit = function() { this.last_req = this.request("POST", this.uri + "_ensure_full_commit"); CouchDB.maybeThrowError(this.last_req); return JSON.parse(this.last_req.responseText); - } + }; // Applies the map function to the contents of database and returns the results. this.query = function(mapFun, reduceFun, options, keys, language) { @@ -163,7 +163,7 @@ function CouchDB(name, httpHeaders) { }); CouchDB.maybeThrowError(this.last_req); return JSON.parse(this.last_req.responseText); - } + }; this.view = function(viewname, options, keys) { var viewParts = viewname.split('/'); @@ -182,21 +182,21 @@ function CouchDB(name, httpHeaders) { } CouchDB.maybeThrowError(this.last_req); return JSON.parse(this.last_req.responseText); - } + }; // gets information about the database this.info = function() { this.last_req = this.request("GET", this.uri); CouchDB.maybeThrowError(this.last_req); return JSON.parse(this.last_req.responseText); - } + }; // gets information about a design doc this.designInfo = function(docid) { this.last_req = this.request("GET", this.uri + docid + "/_info"); CouchDB.maybeThrowError(this.last_req); return JSON.parse(this.last_req.responseText); - } + }; this.allDocs = function(options,keys) { if(!keys) { @@ -211,7 +211,7 @@ function CouchDB(name, httpHeaders) { } CouchDB.maybeThrowError(this.last_req); return JSON.parse(this.last_req.responseText); - } + }; this.designDocs = function() { return this.allDocs({startkey:"_design", endkey:"_design0"}); @@ -222,19 +222,19 @@ function CouchDB(name, httpHeaders) { + encodeOptions(options)); CouchDB.maybeThrowError(this.last_req); return JSON.parse(this.last_req.responseText); - } + }; this.compact = function() { this.last_req = this.request("POST", this.uri + "_compact"); CouchDB.maybeThrowError(this.last_req); return JSON.parse(this.last_req.responseText); - } + }; this.viewCleanup = function() { this.last_req = this.request("POST", this.uri + "_view_cleanup"); CouchDB.maybeThrowError(this.last_req); return JSON.parse(this.last_req.responseText); - } + }; this.setDbProperty = function(propId, propValue) { this.last_req = this.request("PUT", this.uri + propId,{ @@ -242,13 +242,13 @@ function CouchDB(name, httpHeaders) { }); CouchDB.maybeThrowError(this.last_req); return JSON.parse(this.last_req.responseText); - } + }; this.getDbProperty = function(propId) { this.last_req = this.request("GET", this.uri + propId); CouchDB.maybeThrowError(this.last_req); return JSON.parse(this.last_req.responseText); - } + }; this.setSecObj = function(secObj) { this.last_req = this.request("PUT", this.uri + "_security",{ @@ -256,21 +256,21 @@ function CouchDB(name, httpHeaders) { }); CouchDB.maybeThrowError(this.last_req); return JSON.parse(this.last_req.responseText); - } + }; this.getSecObj = function() { this.last_req = this.request("GET", this.uri + "_security"); CouchDB.maybeThrowError(this.last_req); return JSON.parse(this.last_req.responseText); - } + }; // Convert a options object to an url query string. // ex: {key:'value',key2:'value2'} becomes '?key="value"&key2="value2"' function encodeOptions(options) { - var buf = [] + var buf = []; if (typeof(options) == "object" && options !== null) { for (var name in options) { - if (!options.hasOwnProperty(name)) { continue }; + if (!options.hasOwnProperty(name)) { continue; }; var value = options[name]; if (name == "key" || name == "startkey" || name == "endkey") { value = toJSON(value); @@ -318,7 +318,7 @@ CouchDB.login = function(name, password) { + encodeURIComponent(password) }); return JSON.parse(CouchDB.last_req.responseText); -} +}; CouchDB.logout = function() { CouchDB.last_req = CouchDB.request("DELETE", "/_session", { @@ -326,7 +326,7 @@ CouchDB.logout = function() { "X-CouchDB-WWW-Authenticate": "Cookie"} }); return JSON.parse(CouchDB.last_req.responseText); -} +}; CouchDB.session = function(options) { options = options || {}; @@ -346,7 +346,7 @@ CouchDB.prepareUserDoc = function(user_doc, new_password) { } user_doc.type = "user"; if (!user_doc.roles) { - user_doc.roles = [] + user_doc.roles = []; } return user_doc; }; @@ -370,7 +370,7 @@ CouchDB.getVersion = function() { CouchDB.last_req = CouchDB.request("GET", "/"); CouchDB.maybeThrowError(CouchDB.last_req); return JSON.parse(CouchDB.last_req.responseText).version; -} +}; CouchDB.replicate = function(source, target, rep_options) { rep_options = rep_options || {}; @@ -384,7 +384,7 @@ CouchDB.replicate = function(source, target, rep_options) { }); CouchDB.maybeThrowError(CouchDB.last_req); return JSON.parse(CouchDB.last_req.responseText); -} +}; CouchDB.newXhr = function() { if (typeof(XMLHttpRequest) != "undefined") { @@ -394,16 +394,16 @@ CouchDB.newXhr = function() { } else { throw new Error("No XMLHTTPRequest support detected"); } -} +}; CouchDB.request = function(method, uri, options) { - options = options || {}; - options.headers = options.headers || {}; + options = typeof(options) == 'object' ? options : {}; + options.headers = typeof(options.headers) == 'object' ? options.headers : {}; options.headers["Content-Type"] = options.headers["Content-Type"] || options.headers["content-type"] || "application/json"; options.headers["Accept"] = options.headers["Accept"] || options.headers["accept"] || "application/json"; var req = CouchDB.newXhr(); if(uri.substr(0, "http://".length) != "http://") { - uri = CouchDB.urlPrefix + uri + uri = CouchDB.urlPrefix + uri; } req.open(method, uri, false); if (options.headers) { @@ -415,7 +415,7 @@ CouchDB.request = function(method, uri, options) { } req.send(options.body || ""); return req; -} +}; CouchDB.requestStats = function(module, key, test) { var query_arg = ""; @@ -426,7 +426,7 @@ CouchDB.requestStats = function(module, key, test) { var url = "/_stats/" + module + "/" + key + query_arg; var stat = CouchDB.request("GET", url).responseText; return JSON.parse(stat)[module][key]; -} +}; CouchDB.uuids_cache = []; @@ -449,7 +449,7 @@ CouchDB.newUuids = function(n, buf) { CouchDB.uuids_cache.concat(result.uuids.slice(0, buf)); return result.uuids.slice(buf); } -} +}; CouchDB.maybeThrowError = function(req) { if (req.status >= 400) { @@ -460,7 +460,7 @@ CouchDB.maybeThrowError = function(req) { } throw result; } -} +}; CouchDB.params = function(options) { options = options || {}; diff --git a/rel/overlay/share/www/script/couch_test_runner.js b/rel/overlay/share/www/script/couch_test_runner.js index 451a454a..55a6533f 100644 --- a/rel/overlay/share/www/script/couch_test_runner.js +++ b/rel/overlay/share/www/script/couch_test_runner.js @@ -14,6 +14,13 @@ function loadScript(url) { + // disallow loading remote URLs + if((url.substr(0, 7) == "http://") + || (url.substr(0, 2) == "//") + || (url.substr(0, 5) == "data:") + || (url.substr(0, 11) == "javascript:")) { + throw "Not loading remote test scripts"; + } if (typeof document != "undefined") document.write('<script src="'+url+'"></script>'); }; @@ -21,7 +28,7 @@ function patchTest(fun) { var source = fun.toString(); var output = ""; var i = 0; - var testMarker = "T(" + var testMarker = "T("; while (i < source.length) { var testStart = source.indexOf(testMarker, i); if (testStart == -1) { @@ -232,13 +239,13 @@ function saveTestReport(report) { $.couch.info({success : function(node_info) { report.node = node_info; db.saveDoc(report); - }}) + }}); }; var createDb = function() { db.create({success: function() { db.info({success:saveReport}); }}); - } + }; db.info({error: createDb, success:saveReport}); } }; @@ -302,7 +309,7 @@ function T(arg1, arg2, testName) { .find("code").text(message).end() .appendTo($("td.details ol", currentRow)); } - numFailures += 1 + numFailures += 1; } } @@ -311,6 +318,11 @@ function TEquals(expected, actual, testName) { "', got '" + repr(actual) + "'", testName); } +function TEqualsIgnoreCase(expected, actual, testName) { + T(equals(expected.toUpperCase(), actual.toUpperCase()), "expected '" + repr(expected) + + "', got '" + repr(actual) + "'", testName); +} + function equals(a,b) { if (a === b) return true; try { @@ -331,18 +343,18 @@ function repr(val) { } function makeDocs(start, end, templateDoc) { - var templateDocSrc = templateDoc ? JSON.stringify(templateDoc) : "{}" + var templateDocSrc = templateDoc ? JSON.stringify(templateDoc) : "{}"; if (end === undefined) { end = start; start = 0; } - var docs = [] + var docs = []; for (var i = start; i < end; i++) { var newDoc = eval("(" + templateDocSrc + ")"); newDoc._id = (i).toString(); newDoc.integer = i; newDoc.string = (i).toString(); - docs.push(newDoc) + docs.push(newDoc); } return docs; } diff --git a/rel/overlay/share/www/script/couch_tests.js b/rel/overlay/share/www/script/couch_tests.js index c5257ea6..896b3538 100644 --- a/rel/overlay/share/www/script/couch_tests.js +++ b/rel/overlay/share/www/script/couch_tests.js @@ -32,6 +32,7 @@ loadTest("basics.js"); loadTest("all_docs.js"); loadTest("attachments.js"); loadTest("attachments_multipart.js"); +loadTest("attachment_conflicts.js"); loadTest("attachment_names.js"); loadTest("attachment_paths.js"); loadTest("attachment_views.js"); diff --git a/rel/overlay/share/www/script/futon.browse.js b/rel/overlay/share/www/script/futon.browse.js index c8c1c420..a3f6e8cb 100644 --- a/rel/overlay/share/www/script/futon.browse.js +++ b/rel/overlay/share/www/script/futon.browse.js @@ -97,7 +97,10 @@ // Page class for browse/database.html CouchDatabasePage: function() { var urlParts = location.search.substr(1).split("/"); - var dbName = decodeURIComponent(urlParts.shift()); + var dbName = decodeURIComponent(urlParts.shift()) + + var dbNameRegExp = new RegExp("[^a-z0-9\_\$\(\)\+\/\-]", "g"); + dbName = dbName.replace(dbNameRegExp, ""); $.futon.storage.declareWithPrefix(dbName + ".", { desc: {}, @@ -113,18 +116,19 @@ var viewName = (urlParts.length > 0) ? urlParts.join("/") : null; if (viewName) { - $.futon.storage.set("view", viewName); + $.futon.storage.set("view", decodeURIComponent(viewName)); } else { viewName = $.futon.storage.get("view"); if (viewName) { this.redirecting = true; location.href = "database.html?" + encodeURIComponent(dbName) + - "/" + viewName; + "/" + encodeURIComponent(viewName); } } var db = $.couch.db(dbName); this.dbName = dbName; + viewName = decodeURIComponent(viewName); this.viewName = viewName; this.viewLanguage = "javascript"; this.db = db; @@ -150,9 +154,13 @@ db.compact({success: function(resp) { callback() }}); break; case "compact_views": - var groupname = page.viewName.substring(8, - page.viewName.indexOf("/_view")); - db.compactView(groupname, {success: function(resp) { callback() }}); + var idx = page.viewName.indexOf("/_view"); + if (idx == -1) { + alert("Compact Views requires focus on a view!"); + } else { + var groupname = page.viewName.substring(8, idx); + db.compactView(groupname, {success: function(resp) { callback() }}); + } break; case "view_cleanup": db.viewCleanup({success: function(resp) { callback() }}); @@ -372,7 +380,8 @@ var path = $.couch.encodeDocId(doc._id) + "/_view/" + encodeURIComponent(viewNames[j]); var option = $(document.createElement("option")) - .attr("value", path).text(viewNames[j]).appendTo(optGroup); + .attr("value", path).text(encodeURIComponent(viewNames[j])) + .appendTo(optGroup); if (path == viewName) { option[0].selected = true; } @@ -408,7 +417,7 @@ } var viewCode = resp.views[localViewName]; page.viewLanguage = resp.language || "javascript"; - $("#language").val(page.viewLanguage); + $("#language").val(encodeURIComponent(page.viewLanguage)); page.updateViewEditor(viewCode.map, viewCode.reduce || ""); $("#viewcode button.revert, #viewcode button.save").attr("disabled", "disabled"); page.storedViewCode = viewCode; @@ -420,7 +429,7 @@ page.updateViewEditor(page.storedViewCode.map, page.storedViewCode.reduce || ""); page.viewLanguage = page.storedViewLanguage; - $("#language").val(page.viewLanguage); + $("#language").val(encodeURIComponent(page.viewLanguage)); $("#viewcode button.revert, #viewcode button.save").attr("disabled", "disabled"); page.isDirty = false; if (callback) callback(); @@ -504,7 +513,8 @@ callback({ docid: "Cannot save to " + data.docid + " because its language is \"" + doc.language + - "\", not \"" + page.viewLanguage + "\"." + "\", not \"" + + encodeURIComponent(page.viewLanguage) + "\"." }); return; } @@ -569,7 +579,7 @@ this.updateDesignDocLink = function() { if (viewName && /^_design/.test(viewName)) { - var docId = "_design/" + decodeURIComponent(viewName.split("/")[1]); + var docId = "_design/" + encodeURIComponent(decodeURIComponent(viewName).split("/")[1]); $("#designdoc-link").attr("href", "document.html?" + encodeURIComponent(dbName) + "/" + $.couch.encodeDocId(docId)).text(docId); } else { @@ -765,8 +775,7 @@ if (page.isDirty) { db.query(currentMapCode, currentReduceCode, page.viewLanguage, options); } else { - var viewParts = viewName.split('/'); - + var viewParts = decodeURIComponent(viewName).split('/'); if ($.futon.storage.get("stale")) { options.stale = "ok"; } diff --git a/rel/overlay/share/www/script/futon.format.js b/rel/overlay/share/www/script/futon.format.js index 0d536e36..8d9b7f5c 100644 --- a/rel/overlay/share/www/script/futon.format.js +++ b/rel/overlay/share/www/script/futon.format.js @@ -16,7 +16,10 @@ escape: function(string) { return string.replace(/&/g, "&") .replace(/</g, "<") - .replace(/>/g, ">"); + .replace(/>/g, ">") + .replace(/"/, """) + .replace(/'/, "'") + ; }, // JSON pretty printing diff --git a/rel/overlay/share/www/script/futon.js b/rel/overlay/share/www/script/futon.js index b15a5eec..c4647ed1 100644 --- a/rel/overlay/share/www/script/futon.js +++ b/rel/overlay/share/www/script/futon.js @@ -215,9 +215,10 @@ function $$(node) { recentDbs.sort(); $.each(recentDbs, function(idx, name) { if (name) { + name = encodeURIComponent(name); $("#dbs").append("<li>" + "<button class='remove' title='Remove from list' value='" + name + "'></button>" + - "<a href='database.html?" + encodeURIComponent(name) + "' title='" + name + "'>" + name + + "<a href='database.html?" + name + "' title='" + name + "'>" + name + "</a></li>"); } }); @@ -334,6 +335,14 @@ function $$(node) { return callback(decl); } + function windowName() { + try { + return JSON.parse(window.name || "{}"); + } catch (e) { + return {}; + } + } + // add suffix to cookie names to be able to separate between ports var cookiePrefix = location.port + "_"; @@ -366,15 +375,15 @@ function $$(node) { "window": { get: function(name) { - return JSON.parse(window.name || "{}")[name]; + return windowName()[name]; }, set: function(name, value) { - var obj = JSON.parse(window.name || "{}"); + var obj = windowName(); obj[name] = value || null; window.name = JSON.stringify(obj); }, del: function(name) { - var obj = JSON.parse(window.name || "{}"); + var obj = windowName(); delete obj[name]; window.name = JSON.stringify(obj); } diff --git a/rel/overlay/share/www/script/jquery.couch.js b/rel/overlay/share/www/script/jquery.couch.js index ebf7d52a..114e5801 100644 --- a/rel/overlay/share/www/script/jquery.couch.js +++ b/rel/overlay/share/www/script/jquery.couch.js @@ -36,7 +36,7 @@ } user_doc.type = "user"; if (!user_doc.roles) { - user_doc.roles = [] + user_doc.roles = []; } return user_doc; }; @@ -75,7 +75,7 @@ req.type = "PUT"; req.data = toJSON(value); req.contentType = "application/json"; - req.processData = false + req.processData = false; } ajax(req, options, @@ -115,7 +115,7 @@ user_doc = prepareUserDoc(user_doc, password); $.couch.userDb(function(db) { db.saveDoc(user_doc, options); - }) + }); }, login: function(options) { @@ -167,7 +167,7 @@ doc._attachments["rev-"+doc._rev.split("-")[0]] = { content_type :"application/json", data : Base64.encode(rawDocs[doc._id].raw) - } + }; return true; } } @@ -583,7 +583,7 @@ if (!uuidCache.length) { ajax({url: this.urlPrefix + "/_uuids", data: {count: cacheNum}, async: false}, { success: function(resp) { - uuidCache = resp.uuids + uuidCache = resp.uuids; } }, "Failed to retrieve UUID batch." diff --git a/rel/overlay/share/www/script/test/attachment_names.js b/rel/overlay/share/www/script/test/attachment_names.js index d90c24c4..988dd2d2 100644 --- a/rel/overlay/share/www/script/test/attachment_names.js +++ b/rel/overlay/share/www/script/test/attachment_names.js @@ -24,7 +24,7 @@ couchTests.attachment_names = function(debug) { data: "VGhpcyBpcyBhIGJhc2U2NCBlbmNvZGVkIHRleHQ=" } } - } + }; // inline attachments try { @@ -72,7 +72,7 @@ couchTests.attachment_names = function(debug) { data: "VGhpcyBpcyBhIGJhc2U2NCBlbmNvZGVkIHRleHQ=" } } - } + }; try { db.save(binAttDoc); diff --git a/rel/overlay/share/www/script/test/attachment_paths.js b/rel/overlay/share/www/script/test/attachment_paths.js index a2a0f69c..3f6ffb7c 100644 --- a/rel/overlay/share/www/script/test/attachment_paths.js +++ b/rel/overlay/share/www/script/test/attachment_paths.js @@ -33,7 +33,7 @@ couchTests.attachment_paths = function(debug) { data: "V2UgbGlrZSBwZXJjZW50IHR3byBGLg==" } } - } + }; T(db.save(binAttDoc).ok); @@ -73,7 +73,10 @@ couchTests.attachment_paths = function(debug) { T(binAttDoc._attachments["foo/bar.txt"] !== undefined); T(binAttDoc._attachments["foo%2Fbaz.txt"] !== undefined); T(binAttDoc._attachments["foo/bar2.txt"] !== undefined); - T(binAttDoc._attachments["foo/bar2.txt"].content_type == "text/plain;charset=utf-8"); + TEquals("text/plain;charset=utf-8", // thank you Safari + binAttDoc._attachments["foo/bar2.txt"].content_type.toLowerCase(), + "correct content-type" + ); T(binAttDoc._attachments["foo/bar2.txt"].length == 30); //// now repeat the while thing with a design doc @@ -92,7 +95,7 @@ couchTests.attachment_paths = function(debug) { data: "V2UgbGlrZSBwZXJjZW50IHR3byBGLg==" } } - } + }; T(db.save(binAttDoc).ok); @@ -141,7 +144,10 @@ couchTests.attachment_paths = function(debug) { T(binAttDoc._attachments["foo/bar.txt"] !== undefined); T(binAttDoc._attachments["foo/bar2.txt"] !== undefined); - T(binAttDoc._attachments["foo/bar2.txt"].content_type == "text/plain;charset=utf-8"); + TEquals("text/plain;charset=utf-8", // thank you Safari + binAttDoc._attachments["foo/bar2.txt"].content_type.toLowerCase(), + "correct content-type" + ); T(binAttDoc._attachments["foo/bar2.txt"].length == 30); } }; diff --git a/rel/overlay/share/www/script/test/attachment_views.js b/rel/overlay/share/www/script/test/attachment_views.js index fd30dcfc..a92a8ad0 100644 --- a/rel/overlay/share/www/script/test/attachment_views.js +++ b/rel/overlay/share/www/script/test/attachment_views.js @@ -68,11 +68,11 @@ couchTests.attachment_views= function(debug) { } emit(parseInt(doc._id), count); - } + }; var reduceFunction = function(key, values) { return sum(values); - } + }; var result = db.query(mapFunction, reduceFunction); diff --git a/rel/overlay/share/www/script/test/attachments.js b/rel/overlay/share/www/script/test/attachments.js index 9d89d5d0..e16c384f 100644 --- a/rel/overlay/share/www/script/test/attachments.js +++ b/rel/overlay/share/www/script/test/attachments.js @@ -24,7 +24,7 @@ couchTests.attachments= function(debug) { data: "VGhpcyBpcyBhIGJhc2U2NCBlbmNvZGVkIHRleHQ=" } } - } + }; var save_response = db.save(binAttDoc); T(save_response.ok); @@ -43,7 +43,7 @@ couchTests.attachments= function(debug) { data: "" } } - } + }; T(db.save(binAttDoc2).ok); @@ -68,12 +68,12 @@ couchTests.attachments= function(debug) { T(binAttDoc2._attachments["foo.txt"] !== undefined); T(binAttDoc2._attachments["foo2.txt"] !== undefined); - T(binAttDoc2._attachments["foo2.txt"].content_type == "text/plain;charset=utf-8"); + TEqualsIgnoreCase("text/plain;charset=utf-8", binAttDoc2._attachments["foo2.txt"].content_type); T(binAttDoc2._attachments["foo2.txt"].length == 30); var xhr = CouchDB.request("GET", "/test_suite_db/bin_doc2/foo2.txt"); T(xhr.responseText == "This is no base64 encoded text"); - T(xhr.getResponseHeader("Content-Type") == "text/plain;charset=utf-8"); + TEqualsIgnoreCase("text/plain;charset=utf-8", xhr.getResponseHeader("Content-Type")); // test without rev, should fail var xhr = CouchDB.request("DELETE", "/test_suite_db/bin_doc2/foo2.txt"); @@ -96,7 +96,7 @@ couchTests.attachments= function(debug) { var xhr = CouchDB.request("GET", "/test_suite_db/bin_doc3/attachment.txt"); T(xhr.responseText == bin_data); - T(xhr.getResponseHeader("Content-Type") == "text/plain;charset=utf-8"); + TEqualsIgnoreCase("text/plain;charset=utf-8", xhr.getResponseHeader("Content-Type")); var xhr = CouchDB.request("PUT", "/test_suite_db/bin_doc3/attachment.txt", { headers:{"Content-Type":"text/plain;charset=utf-8"}, @@ -113,11 +113,11 @@ couchTests.attachments= function(debug) { var xhr = CouchDB.request("GET", "/test_suite_db/bin_doc3/attachment.txt"); T(xhr.responseText == bin_data); - T(xhr.getResponseHeader("Content-Type") == "text/plain;charset=utf-8"); + TEqualsIgnoreCase("text/plain;charset=utf-8", xhr.getResponseHeader("Content-Type")); var xhr = CouchDB.request("GET", "/test_suite_db/bin_doc3/attachment.txt?rev=" + rev); T(xhr.responseText == bin_data); - T(xhr.getResponseHeader("Content-Type") == "text/plain;charset=utf-8"); + TEqualsIgnoreCase("text/plain;charset=utf-8", xhr.getResponseHeader("Content-Type")); var xhr = CouchDB.request("DELETE", "/test_suite_db/bin_doc3/attachment.txt?rev=" + rev); T(xhr.status == 200); @@ -129,7 +129,7 @@ couchTests.attachments= function(debug) { var xhr = CouchDB.request("GET", "/test_suite_db/bin_doc3/attachment.txt?rev=" + rev); T(xhr.status == 200); T(xhr.responseText == bin_data); - T(xhr.getResponseHeader("Content-Type") == "text/plain;charset=utf-8"); + TEqualsIgnoreCase("text/plain;charset=utf-8", xhr.getResponseHeader("Content-Type")); // empty attachments var xhr = CouchDB.request("PUT", "/test_suite_db/bin_doc4/attachment.txt", { @@ -156,7 +156,7 @@ couchTests.attachments= function(debug) { // Attachment sparseness COUCHDB-220 - var docs = [] + var docs = []; for (var i = 0; i < 5; i++) { var doc = { _id: (i).toString(), @@ -166,8 +166,8 @@ couchTests.attachments= function(debug) { data: "VGhpcyBpcyBhIGJhc2U2NCBlbmNvZGVkIHRleHQ=" } } - } - docs.push(doc) + }; + docs.push(doc); } var saved = db.bulkSave(docs); @@ -210,7 +210,7 @@ couchTests.attachments= function(debug) { var xhr = CouchDB.request("GET", "/test_suite_db/bin_doc5/lorem.txt"); T(xhr.responseText == lorem); - T(xhr.getResponseHeader("Content-Type") == "text/plain;charset=utf-8"); + TEqualsIgnoreCase("text/plain;charset=utf-8", xhr.getResponseHeader("Content-Type")); // test large inline attachment too var lorem_b64 = CouchDB.request("GET", "/_utils/script/test/lorem_b64.txt").responseText; @@ -254,7 +254,7 @@ couchTests.attachments= function(debug) { data: "VGhpcyBpcyBhIGJhc2U2NCBlbmNvZGVkIHRleHQ=" } } - } + }; T(db.save(bin_doc6).ok); // stub out the attachment bin_doc6._attachments["foo.txt"] = { stub: true }; @@ -268,6 +268,6 @@ couchTests.attachments= function(debug) { T(db.save(bin_doc6).ok == true); T(false && "Shouldn't get here!"); } catch (e) { - T(e.error == "missing_stub") + T(e.error == "missing_stub"); } }; diff --git a/rel/overlay/share/www/script/test/attachments_multipart.js b/rel/overlay/share/www/script/test/attachments_multipart.js index 2b79e559..f173d2bb 100644 --- a/rel/overlay/share/www/script/test/attachments_multipart.js +++ b/rel/overlay/share/www/script/test/attachments_multipart.js @@ -29,17 +29,17 @@ couchTests.attachments_multipart= function(debug) { "_attachments":{ "foo.txt": { "follows":true, - "content_type":"text/plain", + "content_type":"application/test", "length":21 }, "bar.txt": { "follows":true, - "content_type":"text/plain", + "content_type":"application/test", "length":20 }, "baz.txt": { "follows":true, - "content_type":"text/plain", + "content_type":"application/test", "length":19 } } @@ -58,7 +58,7 @@ couchTests.attachments_multipart= function(debug) { var result = JSON.parse(xhr.responseText); - T(result.ok) + T(result.ok); @@ -193,7 +193,7 @@ couchTests.attachments_multipart= function(debug) { // a certain rev). xhr = CouchDB.request("GET", "/test_suite_db/multipart?atts_since=[\"" + firstrev + "\"]", - {headers:{"accept": "multipart/related,*/*;"}}); + {headers:{"accept": "multipart/related, */*"}}); T(xhr.status == 200); diff --git a/rel/overlay/share/www/script/test/auth_cache.js b/rel/overlay/share/www/script/test/auth_cache.js index 75827dbd..e48f7370 100644 --- a/rel/overlay/share/www/script/test/auth_cache.js +++ b/rel/overlay/share/www/script/test/auth_cache.js @@ -238,6 +238,37 @@ couchTests.auth_cache = function(debug) { T(misses_after === misses_before); T(hits_after === (hits_before + 1)); + + // login, compact authentication DB, login again and verify that + // there was a cache hit + hits_before = hits_after; + misses_before = misses_after; + + T(CouchDB.login("johndoe", "123456").ok); + + hits_after = hits(); + misses_after = misses(); + + T(misses_after === (misses_before + 1)); + T(hits_after === hits_before); + + T(CouchDB.logout().ok); + T(authDb.compact().ok); + + while (authDb.info().compact_running); + + hits_before = hits_after; + misses_before = misses_after; + + T(CouchDB.login("johndoe", "123456").ok); + + hits_after = hits(); + misses_after = misses(); + + T(misses_after === misses_before); + T(hits_after === (hits_before + 1)); + + T(CouchDB.logout().ok); } diff --git a/rel/overlay/share/www/script/test/basics.js b/rel/overlay/share/www/script/test/basics.js index 6a3ae471..8885ba6e 100644 --- a/rel/overlay/share/www/script/test/basics.js +++ b/rel/overlay/share/www/script/test/basics.js @@ -45,7 +45,7 @@ couchTests.basics = function(debug) { // Get the database info, check the db_name T(db.info().db_name == "test_suite_db"); - T(CouchDB.allDbs().indexOf("test_suite_db") != -1) + T(CouchDB.allDbs().indexOf("test_suite_db") != -1); // Get the database info, check the doc_count T(db.info().doc_count == 0); @@ -91,13 +91,13 @@ couchTests.basics = function(debug) { emit(null, doc.b); }; - results = db.query(mapFunction); + var results = db.query(mapFunction); // verify only one document found and the result value (doc.b). T(results.total_rows == 1 && results.rows[0].value == 16); // reopen document we saved earlier - existingDoc = db.open(id); + var existingDoc = db.open(id); T(existingDoc.a==1); @@ -191,12 +191,12 @@ couchTests.basics = function(debug) { T(xhr.status == 404); // Check for invalid document members - bad_docs = [ + var bad_docs = [ ["goldfish", {"_zing": 4}], ["zebrafish", {"_zoom": "hello"}], ["mudfish", {"zane": "goldfish", "_fan": "something smells delicious"}], ["tastyfish", {"_bing": {"wha?": "soda can"}}] - ] + ]; var test_doc = function(info) { var data = JSON.stringify(info[1]); xhr = CouchDB.request("PUT", "/test_suite_db/" + info[0], {body: data}); diff --git a/rel/overlay/share/www/script/test/bulk_docs.js b/rel/overlay/share/www/script/test/bulk_docs.js index 346aea83..9095e6b3 100644 --- a/rel/overlay/share/www/script/test/bulk_docs.js +++ b/rel/overlay/share/www/script/test/bulk_docs.js @@ -51,12 +51,12 @@ couchTests.bulk_docs = function(debug) { T(results.length == 5); T(results[0].id == "0"); T(results[0].error == "conflict"); - T(results[0].rev === undefined); // no rev member when a conflict + T(typeof results[0].rev === "undefined"); // no rev member when a conflict // but the rest are not for (i = 1; i < 5; i++) { T(results[i].id == i.toString()); - T(results[i].rev) + T(results[i].rev); T(db.open(docs[i]._id) == null); } @@ -64,7 +64,7 @@ couchTests.bulk_docs = function(debug) { // save doc 0, this will cause a conflict when we save docs[0] var doc = db.open("0"); - docs[0] = db.open("0") + docs[0] = db.open("0"); db.save(doc); docs[0].shooby = "dooby"; @@ -93,8 +93,8 @@ couchTests.bulk_docs = function(debug) { // Regression test for failure on update/delete var newdoc = {"_id": "foobar", "body": "baz"}; T(db.save(newdoc).ok); - update = {"_id": newdoc._id, "_rev": newdoc._rev, "body": "blam"}; - torem = {"_id": newdoc._id, "_rev": newdoc._rev, "_deleted": true}; + var update = {"_id": newdoc._id, "_rev": newdoc._rev, "body": "blam"}; + var torem = {"_id": newdoc._id, "_rev": newdoc._rev, "_deleted": true}; results = db.bulkSave([update, torem]); T(results[0].error == "conflict" || results[1].error == "conflict"); }; diff --git a/rel/overlay/share/www/script/test/compact.js b/rel/overlay/share/www/script/test/compact.js index 22eeaec1..805a3b08 100644 --- a/rel/overlay/share/www/script/test/compact.js +++ b/rel/overlay/share/www/script/test/compact.js @@ -26,7 +26,7 @@ couchTests.compact = function(debug) { data: "VGhpcyBpcyBhIGJhc2U2NCBlbmNvZGVkIHRleHQ=" } } - } + }; T(db.save(binAttDoc).ok); @@ -51,8 +51,8 @@ couchTests.compact = function(debug) { T(db.ensureFullCommit().ok); restartServer(); var xhr = CouchDB.request("GET", "/test_suite_db/bin_doc/foo.txt"); - T(xhr.responseText == "This is a base64 encoded text") - T(xhr.getResponseHeader("Content-Type") == "text/plain") + T(xhr.responseText == "This is a base64 encoded text"); + T(xhr.getResponseHeader("Content-Type") == "text/plain"); T(db.info().doc_count == 1); T(db.info().disk_size < deletesize); diff --git a/rel/overlay/share/www/script/test/conflicts.js b/rel/overlay/share/www/script/test/conflicts.js index b8b93946..7258bc31 100644 --- a/rel/overlay/share/www/script/test/conflicts.js +++ b/rel/overlay/share/www/script/test/conflicts.js @@ -44,7 +44,7 @@ couchTests.conflicts = function(debug) { var changes = db.changes(); - T( changes.results.length == 1) + T(changes.results.length == 1); // Now clear out the _rev member and save. This indicates this document is // new, not based on an existing revision. diff --git a/rel/overlay/share/www/script/test/cookie_auth.js b/rel/overlay/share/www/script/test/cookie_auth.js index 68ec882d..ef915602 100644 --- a/rel/overlay/share/www/script/test/cookie_auth.js +++ b/rel/overlay/share/www/script/test/cookie_auth.js @@ -65,7 +65,7 @@ couchTests.cookie_auth = function(debug) { }, "eh, Boo-Boo?"); try { - usersDb.save(duplicateJchrisDoc) + usersDb.save(duplicateJchrisDoc); T(false && "Can't create duplicate user names. Should have thrown an error."); } catch (e) { T(e.error == "conflict"); @@ -78,7 +78,7 @@ couchTests.cookie_auth = function(debug) { }, "copperfield"); try { - usersDb.save(underscoreUserDoc) + usersDb.save(underscoreUserDoc); T(false && "Can't create underscore user names. Should have thrown an error."); } catch (e) { T(e.error == "forbidden"); @@ -93,7 +93,7 @@ couchTests.cookie_auth = function(debug) { badIdDoc._id = "org.apache.couchdb:w00x"; try { - usersDb.save(badIdDoc) + usersDb.save(badIdDoc); T(false && "Can't create malformed docids. Should have thrown an error."); } catch (e) { T(e.error == "forbidden"); @@ -125,7 +125,7 @@ couchTests.cookie_auth = function(debug) { T(CouchDB.session().userCtx.name != 'Jason Davies'); // test redirect - xhr = CouchDB.request("POST", "/_session?next=/", { + var xhr = CouchDB.request("POST", "/_session?next=/", { headers: {"Content-Type": "application/x-www-form-urlencoded"}, body: "name=Jason%20Davies&password="+encodeURIComponent(password) }); @@ -135,10 +135,10 @@ couchTests.cookie_auth = function(debug) { // to follow the redirect, ie, the browser follows and does a // GET on the returned Location if (xhr.status == 200) { - T(/Welcome/.test(xhr.responseText)) + T(/Welcome/.test(xhr.responseText)); } else { - T(xhr.status == 302) - T(xhr.getResponseHeader("Location")) + T(xhr.status == 302); + T(xhr.getResponseHeader("Location")); } // test users db validations @@ -151,7 +151,7 @@ couchTests.cookie_auth = function(debug) { jasonUserDoc.foo=3; try { - usersDb.save(jasonUserDoc) + usersDb.save(jasonUserDoc); T(false && "Can't update someone else's user doc. Should have thrown an error."); } catch (e) { T(e.error == "forbidden"); @@ -162,7 +162,7 @@ couchTests.cookie_auth = function(debug) { jchrisUserDoc.roles = ["foo"]; try { - usersDb.save(jchrisUserDoc) + usersDb.save(jchrisUserDoc); T(false && "Can't set roles unless you are admin. Should have thrown an error."); } catch (e) { T(e.error == "forbidden"); @@ -179,7 +179,7 @@ couchTests.cookie_auth = function(debug) { jchrisUserDoc.roles = ["_bar"]; try { - usersDb.save(jchrisUserDoc) + usersDb.save(jchrisUserDoc); T(false && "Can't add system roles to user's db. Should have thrown an error."); } catch (e) { T(e.error == "forbidden"); diff --git a/rel/overlay/share/www/script/test/design_docs.js b/rel/overlay/share/www/script/test/design_docs.js index e62951ac..a24167b2 100644 --- a/rel/overlay/share/www/script/test/design_docs.js +++ b/rel/overlay/share/www/script/test/design_docs.js @@ -41,8 +41,8 @@ function() { whatever : { stringzone : "exports.string = 'plankton';", commonjs : { - whynot : "exports.test = require('../stringzone')", - upper : "exports.testing = require('./whynot').test.string.toUpperCase()+module.id" + whynot : "exports.test = require('../stringzone'); exports.foo = require('whatever/stringzone');", + upper : "exports.testing = require('./whynot').test.string.toUpperCase()+module.id+require('./whynot').foo.string" } }, views: { @@ -58,7 +58,8 @@ function() { }, shows: { simple: "function() {return 'ok'};", - requirey : "function() { var lib = require('whatever/commonjs/upper'); return lib.testing; };" + requirey : "function() { var lib = require('whatever/commonjs/upper'); return lib.testing; };", + circular : "function() { var lib = require('whatever/commonjs/upper'); return JSON.stringify(this); };" } }; @@ -86,7 +87,15 @@ function() { // test commonjs require var xhr = CouchDB.request("GET", "/test_suite_db/_design/test/_show/requirey"); T(xhr.status == 200); - TEquals("PLANKTONwhatever/commonjs/upper", xhr.responseText); + TEquals("PLANKTONwhatever/commonjs/upperplankton", xhr.responseText); + + var xhr = CouchDB.request("GET", "/test_suite_db/_design/test/_show/circular"); + T(xhr.status == 200); + TEquals("javascript", JSON.parse(xhr.responseText).language); + + var prev_view_sig = db.designInfo("_design/test").view_index.signature; + + db.bulkSave(makeDocs(1, numDocs + 1)); // test that we get design doc info back var dinfo = db.designInfo("_design/test"); @@ -94,9 +103,27 @@ function() { var vinfo = dinfo.view_index; TEquals(51, vinfo.disk_size); TEquals(false, vinfo.compact_running); - TEquals("3f88e53b303e2342e49a66c538c30679", vinfo.signature); + // test that GET /db/_design/test/_info + // hasn't triggered an update of the views + TEquals(prev_view_sig, vinfo.signature, 'ddoc sig'); + for (var loop = 0; loop < 2; loop++) { + T(db.view("test/all_docs_twice", {stale: "ok"}).total_rows === 0); + T(db.view("test/single_doc", {stale: "ok"}).total_rows === 0); + T(db.view("test/summate", {stale: "ok"}).rows.length === 0); + T(db.ensureFullCommit().ok); + restartServer(); + }; - db.bulkSave(makeDocs(1, numDocs + 1)); + // test that POST /db/_view_cleanup + // doesn't trigger an update of the views + T(db.viewCleanup().ok); + for (var loop = 0; loop < 2; loop++) { + T(db.view("test/all_docs_twice", {stale: "ok"}).total_rows == 0); + T(db.view("test/single_doc", {stale: "ok"}).total_rows == 0); + T(db.view("test/summate", {stale: "ok"}).rows.length == 0); + T(db.ensureFullCommit().ok); + restartServer(); + }; // test that the _all_docs view returns correctly with keys var results = db.allDocs({startkey:"_design", endkey:"_design0"}); @@ -107,9 +134,9 @@ function() { for (var i = 0; i < numDocs; i++) { T(rows[2*i].key == i+1); T(rows[(2*i)+1].key == i+1); - } - T(db.view("test/no_docs").total_rows == 0) - T(db.view("test/single_doc").total_rows == 1) + }; + T(db.view("test/no_docs").total_rows == 0); + T(db.view("test/single_doc").total_rows == 1); T(db.ensureFullCommit().ok); restartServer(); }; diff --git a/rel/overlay/share/www/script/test/erlang_views.js b/rel/overlay/share/www/script/test/erlang_views.js index 5e93cb96..7eddab40 100644 --- a/rel/overlay/share/www/script/test/erlang_views.js +++ b/rel/overlay/share/www/script/test/erlang_views.js @@ -44,7 +44,7 @@ couchTests.erlang_views = function(debug) { // check simple reduction - another doc with same key. var doc = {_id: "2", integer: 1, string: "str2"}; T(db.save(doc).ok); - rfun = "fun(Keys, Values, ReReduce) -> length(Values) end." + rfun = "fun(Keys, Values, ReReduce) -> length(Values) end."; results = db.query(mfun, rfun, null, null, "erlang"); T(results.rows[0].value == 2); diff --git a/rel/overlay/share/www/script/test/etags_views.js b/rel/overlay/share/www/script/test/etags_views.js index a12734f8..7e1537bd 100644 --- a/rel/overlay/share/www/script/test/etags_views.js +++ b/rel/overlay/share/www/script/test/etags_views.js @@ -38,7 +38,7 @@ couchTests.etags_views = function(debug) { }) } } - } + }; T(db.save(designDoc).ok); var xhr; var docs = makeDocs(0, 10); diff --git a/rel/overlay/share/www/script/test/list_views.js b/rel/overlay/share/www/script/test/list_views.js index f826b46f..44afa899 100644 --- a/rel/overlay/share/www/script/test/list_views.js +++ b/rel/overlay/share/www/script/test/list_views.js @@ -394,7 +394,7 @@ couchTests.list_views = function(debug) { T(/LastKey: 0/.test(xhr.responseText)); // Test we do multi-key requests on lists and views in separate docs. - var url = "/test_suite_db/_design/lists/_list/simpleForm/views/basicView" + var url = "/test_suite_db/_design/lists/_list/simpleForm/views/basicView"; xhr = CouchDB.request("POST", url, { body: '{"keys":[-2,-4,-5,-7]}' }); diff --git a/rel/overlay/share/www/script/test/method_override.js b/rel/overlay/share/www/script/test/method_override.js index 26e9bee0..0bb4c61f 100644 --- a/rel/overlay/share/www/script/test/method_override.js +++ b/rel/overlay/share/www/script/test/method_override.js @@ -28,7 +28,7 @@ couchTests.method_override = function(debug) { T(doc.bob == "connie"); xhr = CouchDB.request("POST", "/test_suite_db/fnord?rev=" + doc._rev, {headers:{"X-HTTP-Method-Override" : "DELETE"}}); - T(xhr.status == 200) + T(xhr.status == 200); xhr = CouchDB.request("GET", "/test_suite_db/fnord2", {body: JSON.stringify(doc), headers:{"X-HTTP-Method-Override" : "PUT"}}); // Method Override is ignored when original Method isn't POST diff --git a/rel/overlay/share/www/script/test/proxyauth.js b/rel/overlay/share/www/script/test/proxyauth.js index 171eef37..91e2f221 100644 --- a/rel/overlay/share/www/script/test/proxyauth.js +++ b/rel/overlay/share/www/script/test/proxyauth.js @@ -39,7 +39,7 @@ couchTests.proxyauth = function(debug) { db.createDb(); var benoitcUserDoc = CouchDB.prepareUserDoc({ - name: "benoitc@apache.org", + name: "benoitc@apache.org" }, "test"); T(usersDb.save(benoitcUserDoc).ok); @@ -56,7 +56,7 @@ couchTests.proxyauth = function(debug) { CouchDB.logout(); - headers = { + var headers = { "X-Auth-CouchDB-UserName": "benoitc@apache.org", "X-Auth-CouchDB-Roles": "test", "X-Auth-CouchDB-Token": hex_hmac_sha1(secret, "benoitc@apache.org") @@ -72,14 +72,13 @@ couchTests.proxyauth = function(debug) { }), "role": stringFun(function(doc, req) { return req.userCtx['roles'][0]; - }), + }) } - - } + }; db.save(designDoc); - req = CouchDB.request("GET", "/test_suite_db/_design/test/_show/welcome", + var req = CouchDB.request("GET", "/test_suite_db/_design/test/_show/welcome", {headers: headers}); T(req.responseText == "Welcome benoitc@apache.org"); @@ -87,7 +86,7 @@ couchTests.proxyauth = function(debug) { {headers: headers}); T(req.responseText == "test"); - xhr = CouchDB.request("PUT", "/_config/couch_httpd_auth/proxy_use_secret",{ + var xhr = CouchDB.request("PUT", "/_config/couch_httpd_auth/proxy_use_secret",{ body : JSON.stringify("true"), headers: {"X-Couch-Persist": "false"} }); diff --git a/rel/overlay/share/www/script/test/purge.js b/rel/overlay/share/www/script/test/purge.js index a924c348..f8f45138 100644 --- a/rel/overlay/share/www/script/test/purge.js +++ b/rel/overlay/share/www/script/test/purge.js @@ -30,7 +30,7 @@ couchTests.purge = function(debug) { all_docs_twice: {map: "function(doc) { emit(doc.integer, null); emit(doc.integer, null) }"}, single_doc: {map: "function(doc) { if (doc._id == \"1\") { emit(1, null) }}"} } - } + }; T(db.save(designDoc).ok); @@ -50,7 +50,7 @@ couchTests.purge = function(debug) { // purge the documents var xhr = CouchDB.request("POST", "/test_suite_db/_purge", { - body: JSON.stringify({"1":[doc1._rev], "2":[doc2._rev]}), + body: JSON.stringify({"1":[doc1._rev], "2":[doc2._rev]}) }); T(xhr.status == 200); @@ -76,6 +76,14 @@ couchTests.purge = function(debug) { } T(db.view("test/single_doc").total_rows == 0); + // purge sequences are preserved after compaction (COUCHDB-1021) + T(db.compact().ok); + T(db.last_req.status == 202); + // compaction isn't instantaneous, loop until done + while (db.info().compact_running) {}; + var compactInfo = db.info(); + T(compactInfo.purge_seq == newInfo.purge_seq); + // purge documents twice in a row without loading views // (causes full view rebuilds) @@ -83,13 +91,13 @@ couchTests.purge = function(debug) { var doc4 = db.open("4"); xhr = CouchDB.request("POST", "/test_suite_db/_purge", { - body: JSON.stringify({"3":[doc3._rev]}), + body: JSON.stringify({"3":[doc3._rev]}) }); T(xhr.status == 200); xhr = CouchDB.request("POST", "/test_suite_db/_purge", { - body: JSON.stringify({"4":[doc4._rev]}), + body: JSON.stringify({"4":[doc4._rev]}) }); T(xhr.status == 200); diff --git a/rel/overlay/share/www/script/test/recreate_doc.js b/rel/overlay/share/www/script/test/recreate_doc.js index a6a64ac0..05843558 100644 --- a/rel/overlay/share/www/script/test/recreate_doc.js +++ b/rel/overlay/share/www/script/test/recreate_doc.js @@ -51,7 +51,7 @@ couchTests.recreate_doc = function(debug) { data: "VGhpcyBpcyBhIGJhc2U2NCBlbmNvZGVkIHRleHQ=" } } - } + }; try { // same as before, but with binary db.save(binAttDoc); diff --git a/rel/overlay/share/www/script/test/reduce.js b/rel/overlay/share/www/script/test/reduce.js index 9c80fa7f..979a0292 100644 --- a/rel/overlay/share/www/script/test/reduce.js +++ b/rel/overlay/share/www/script/test/reduce.js @@ -15,14 +15,15 @@ couchTests.reduce = function(debug) { db.deleteDb(); db.createDb(); if (debug) debugger; - var numDocs = 500 + var numDocs = 500; var docs = makeDocs(1,numDocs + 1); db.bulkSave(docs); var summate = function(N) {return (N+1)*N/2;}; var map = function (doc) { emit(doc.integer, doc.integer); - emit(doc.integer, doc.integer)}; + emit(doc.integer, doc.integer); + }; var reduce = function (keys, values) { return sum(values); }; var result = db.query(map, reduce); T(result.rows[0].value == 2*summate(numDocs)); @@ -69,7 +70,7 @@ couchTests.reduce = function(debug) { T(db.info().doc_count == ((i - 1) * 10 * 11) + ((j + 1) * 11)); } - map = function (doc) {emit(doc.keys, 1)}; + map = function (doc) { emit(doc.keys, 1); }; reduce = function (keys, values) { return sum(values); }; var results = db.query(map, reduce, {group:true}); @@ -107,7 +108,7 @@ couchTests.reduce = function(debug) { db.createDb(); - var map = function (doc) {emit(doc.val, doc.val)}; + var map = function (doc) { emit(doc.val, doc.val); }; var reduceCombine = function (keys, values, rereduce) { // This computes the standard deviation of the mapped results var stdDeviation=0.0; diff --git a/rel/overlay/share/www/script/test/reduce_builtin.js b/rel/overlay/share/www/script/test/reduce_builtin.js index d9635688..c9d41fa4 100644 --- a/rel/overlay/share/www/script/test/reduce_builtin.js +++ b/rel/overlay/share/www/script/test/reduce_builtin.js @@ -16,7 +16,7 @@ couchTests.reduce_builtin = function(debug) { db.createDb(); if (debug) debugger; - var numDocs = 500 + var numDocs = 500; var docs = makeDocs(1,numDocs + 1); db.bulkSave(docs); @@ -28,13 +28,14 @@ couchTests.reduce_builtin = function(debug) { acc += i*i; } return acc; - } + }; // this is the same test as the reduce.js test // only we'll let CouchDB run reduce in Erlang var map = function (doc) { emit(doc.integer, doc.integer); - emit(doc.integer, doc.integer)}; + emit(doc.integer, doc.integer); + }; var result = db.query(map, "_sum"); T(result.rows[0].value == 2*summate(numDocs)); @@ -115,7 +116,7 @@ couchTests.reduce_builtin = function(debug) { T(db.info().doc_count == ((i - 1) * 10 * 11) + ((j + 1) * 11)); } - map = function (doc) {emit(doc.keys, 1)}; + map = function (doc) { emit(doc.keys, 1); }; // with emitted values being 1, count should be the same as sum var builtins = ["_sum", "_count"]; diff --git a/rel/overlay/share/www/script/test/replication.js b/rel/overlay/share/www/script/test/replication.js index d2b3164b..7cc1f823 100644 --- a/rel/overlay/share/www/script/test/replication.js +++ b/rel/overlay/share/www/script/test/replication.js @@ -22,14 +22,14 @@ couchTests.replication = function(debug) { target:"test_suite_db_b"}, {source:"http://" + host + "/test_suite_db_a", target:"http://" + host + "/test_suite_db_b"} - ] + ]; var dbA = new CouchDB("test_suite_db_a", {"X-Couch-Full-Commit":"false"}); var dbB = new CouchDB("test_suite_db_b", {"X-Couch-Full-Commit":"false"}); var numDocs = 10; var xhr; for (var testPair = 0; testPair < dbPairs.length; testPair++) { - var A = dbPairs[testPair].source - var B = dbPairs[testPair].target + var A = dbPairs[testPair].source; + var B = dbPairs[testPair].target; dbA.deleteDb(); dbA.createDb(); @@ -41,7 +41,7 @@ couchTests.replication = function(debug) { test_template: new function () { this.init = function(dbA, dbB) { // before anything has happened - } + }; this.afterAB1 = function(dbA, dbB) { // called after replicating src=A tgt=B first time. }; @@ -165,20 +165,20 @@ couchTests.replication = function(debug) { this.afterAB1 = function(dbA, dbB) { var xhr = CouchDB.request("GET", "/test_suite_db_a/bin_doc/foo%2Bbar.txt"); - T(xhr.responseText == "This is a base64 encoded text") + T(xhr.responseText == "This is a base64 encoded text"); xhr = CouchDB.request("GET", "/test_suite_db_b/bin_doc/foo%2Bbar.txt"); - T(xhr.responseText == "This is a base64 encoded text") + T(xhr.responseText == "This is a base64 encoded text"); // and the design-doc xhr = CouchDB.request("GET", "/test_suite_db_a/_design/with_bin/foo%2Bbar.txt"); - T(xhr.responseText == "This is a base64 encoded text") + T(xhr.responseText == "This is a base64 encoded text"); xhr = CouchDB.request("GET", "/test_suite_db_b/_design/with_bin/foo%2Bbar.txt"); - T(xhr.responseText == "This is a base64 encoded text") + T(xhr.responseText == "This is a base64 encoded text"); }; }, @@ -209,8 +209,8 @@ couchTests.replication = function(debug) { var docB = dbB.open("foo", {conflicts: true, deleted_conflicts: true}); // We should have no conflicts this time - T(docA._conflicts === undefined) - T(docB._conflicts === undefined); + T(typeof docA._conflicts === "undefined"); + T(typeof docB._conflicts === "undefined"); // They show up as deleted conflicts instead T(docA._deleted_conflicts[0] == docB._deleted_conflicts[0]); @@ -229,7 +229,7 @@ couchTests.replication = function(debug) { var seqA = result.source_last_seq; T(0 == result.history[0].start_last_seq); - T(result.history[1] === undefined) + T(typeof result.history[1] === "undefined"); for(test in repTests) { if(repTests[test].afterAB1) repTests[test].afterAB1(dbA, dbB); @@ -239,7 +239,7 @@ couchTests.replication = function(debug) { var seqB = result.source_last_seq; T(0 == result.history[0].start_last_seq); - T(result.history[1] === undefined) + T(typeof result.history[1] === "undefined"); for(test in repTests) { if(repTests[test].afterBA1) repTests[test].afterBA1(dbA, dbB); @@ -252,7 +252,7 @@ couchTests.replication = function(debug) { T(seqA < result2.source_last_seq); T(seqA == result2.history[0].start_last_seq); - T(result2.history[1].end_last_seq == seqA) + T(result2.history[1].end_last_seq == seqA); seqA = result2.source_last_seq; @@ -260,11 +260,11 @@ couchTests.replication = function(debug) { if(repTests[test].afterAB2) repTests[test].afterAB2(dbA, dbB); } - result = CouchDB.replicate(B, A) + result = CouchDB.replicate(B, A); T(seqB < result.source_last_seq); T(seqB == result.history[0].start_last_seq); - T(result.history[1].end_last_seq == seqB) + T(result.history[1].end_last_seq == seqB); seqB = result.source_last_seq; @@ -306,21 +306,21 @@ couchTests.replication = function(debug) { var continuousResult = CouchDB.replicate(dbA.name, "test_suite_db_b", { body: {"continuous": true} }); - T(continuousResult.ok) - T(continuousResult._local_id) + T(continuousResult.ok); + T(continuousResult._local_id); var cancelResult = CouchDB.replicate(dbA.name, "test_suite_db_b", { body: {"cancel": true} }); - T(cancelResult.ok) - T(continuousResult._local_id == cancelResult._local_id) + T(cancelResult.ok); + T(continuousResult._local_id == cancelResult._local_id); try { var cancelResult2 = CouchDB.replicate(dbA.name, "test_suite_db_b", { body: {"cancel": true} }); } catch (e) { - T(e.error == "not_found") + T(e.error == "not_found"); } // test replication object option doc_ids @@ -527,4 +527,160 @@ couchTests.replication = function(debug) { T(docFoo4 === null); } + // test for COUCHDB-868 - design docs' attachments not getting replicated + // when doing a pull replication with HTTP basic auth + dbA = new CouchDB("test_suite_db_a"); + dbB = new CouchDB("test_suite_db_b"); + var usersDb = new CouchDB("test_suite_auth"); + var lorem = CouchDB.request( + "GET", "/_utils/script/test/lorem.txt").responseText; + var lorem_b64 = CouchDB.request( + "GET", "/_utils/script/test/lorem_b64.txt").responseText; + + usersDb.deleteDb(); + usersDb.createDb(); + dbA.deleteDb(); + dbA.createDb(); + dbB.deleteDb(); + dbB.createDb(); + + var atts_ddoc = { + _id: "_design/i_have_atts", + language: "javascript" + }; + T(dbA.save(atts_ddoc).ok); + + var rev = atts_ddoc._rev; + var att_1_name = "lorem.txt"; + var att_2_name = "lorem.dat"; + var xhr = CouchDB.request( + "PUT", "/" + dbA.name + "/" + atts_ddoc._id + "/" + att_1_name + "?rev=" + rev, { + headers: {"Content-Type": "text/plain;charset=utf-8"}, + body: lorem + }); + rev = JSON.parse(xhr.responseText).rev; + T(xhr.status === 201); + xhr = CouchDB.request( + "PUT", "/" + dbA.name + "/" + atts_ddoc._id + "/" + att_2_name + "?rev=" + rev, { + headers: {"Content-Type": "application/data"}, + body: lorem_b64 + }); + T(xhr.status === 201); + + var fdmananaUserDoc = CouchDB.prepareUserDoc({ + name: "fdmanana", + roles: ["reader"] + }, "qwerty"); + T(usersDb.save(fdmananaUserDoc).ok); + + T(dbA.setSecObj({ + admins: { + names: [], + roles: ["admin"] + }, + readers: { + names: [], + roles: ["reader"] + } + }).ok); + T(dbB.setSecObj({ + admins: { + names: ["fdmanana"], + roles: [] + } + }).ok); + + var server_config = [ + { + section: "couch_httpd_auth", + key: "authentication_db", + value: usersDb.name + }, + // to prevent admin party mode + { + section: "admins", + key: "joe", + value: "erlang" + } + ]; + + var test_fun = function() { + T(CouchDB.login("fdmanana", "qwerty").ok); + T(CouchDB.session().userCtx.name === "fdmanana"); + T(CouchDB.session().userCtx.roles.indexOf("_admin") === -1); + + var repResult = CouchDB.replicate( + "http://fdmanana:qwerty@" + host + "/" + dbA.name, + dbB.name + ); + T(repResult.ok === true); + T(repResult.history instanceof Array); + T(repResult.history.length === 1); + T(repResult.history[0].docs_written === 1); + T(repResult.history[0].docs_read === 1); + T(repResult.history[0].doc_write_failures === 0); + + var atts_ddoc_copy = dbB.open(atts_ddoc._id); + T(atts_ddoc_copy !== null); + T(typeof atts_ddoc_copy._attachments === "object"); + T(atts_ddoc_copy._attachments !== null); + T(att_1_name in atts_ddoc_copy._attachments); + T(att_2_name in atts_ddoc_copy._attachments); + + var xhr = CouchDB.request("GET", "/" + dbB.name + "/" + atts_ddoc._id + "/" + att_1_name); + T(xhr.status === 200); + T(xhr.responseText === lorem); + + xhr = CouchDB.request("GET", "/" + dbB.name + "/" + atts_ddoc._id + "/" + att_2_name); + T(xhr.status === 200); + T(xhr.responseText === lorem_b64); + + CouchDB.logout(); + T(CouchDB.login("joe", "erlang").ok); + T(dbA.setSecObj({ + admins: { + names: [], + roles: ["bar"] + }, + readers: { + names: [], + roles: ["foo"] + } + }).ok); + T(dbB.deleteDb().ok === true); + T(dbB.createDb().ok === true); + T(dbB.setSecObj({ + admins: { + names: ["fdmanana"], + roles: [] + } + }).ok); + CouchDB.logout(); + + T(CouchDB.login("fdmanana", "qwerty").ok); + T(CouchDB.session().userCtx.name === "fdmanana"); + T(CouchDB.session().userCtx.roles.indexOf("_admin") === -1); + try { + repResult = CouchDB.replicate( + "http://fdmanana:qwerty@" + host + "/" + dbA.name, + dbB.name + ); + T(false, "replication should have failed"); + } catch(x) { + T(x.error === "unauthorized"); + } + + atts_ddoc_copy = dbB.open(atts_ddoc._id); + T(atts_ddoc_copy === null); + + CouchDB.logout(); + T(CouchDB.login("joe", "erlang").ok); + }; + + run_on_modified_server(server_config, test_fun); + + // cleanup + dbA.deleteDb(); + dbB.deleteDb(); + usersDb.deleteDb(); }; diff --git a/rel/overlay/share/www/script/test/rewrite.js b/rel/overlay/share/www/script/test/rewrite.js index 66b33d74..ff2d3822 100644 --- a/rel/overlay/share/www/script/test/rewrite.js +++ b/rel/overlay/share/www/script/test/rewrite.js @@ -365,7 +365,16 @@ couchTests.rewrite = function(debug) { T(result.uuids.length == 1); var first = result.uuids[0]; }); - }); - -}
\ No newline at end of file + + // test invalid rewrites + // string + var ddoc = { + _id: "_design/invalid", + rewrites: "[{\"from\":\"foo\",\"to\":\"bar\"}]" + } + db.save(ddoc); + var res = CouchDB.request("GET", "/test_suite_db/_design/invalid/_rewrite/foo"); + TEquals(400, res.status, "should return 400"); + +} diff --git a/rel/overlay/share/www/script/test/security_validation.js b/rel/overlay/share/www/script/test/security_validation.js index e0ab17d6..dd3b202e 100644 --- a/rel/overlay/share/www/script/test/security_validation.js +++ b/rel/overlay/share/www/script/test/security_validation.js @@ -136,13 +136,20 @@ couchTests.security_validation = function(debug) { doc.foo=2; T(userDb.save(doc).ok); - // Save a document that's missing an author field. - try { - userDb.save({foo:1}); - T(false && "Can't get here. Should have thrown an error 2"); - } catch (e) { - T(e.error == "forbidden"); - T(userDb.last_req.status == 403); + // Save a document that's missing an author field (before and after compaction) + for (var i=0; i<2; i++) { + try { + userDb.save({foo:1}); + T(false && "Can't get here. Should have thrown an error 2"); + } catch (e) { + T(e.error == "forbidden"); + T(userDb.last_req.status == 403); + } + // compact. + T(db.compact().ok); + T(db.last_req.status == 202); + // compaction isn't instantaneous, loop until done + while (db.info().compact_running) {}; } // Now attempt to update the document as a different user, Jan diff --git a/rel/overlay/share/www/script/test/stats.js b/rel/overlay/share/www/script/test/stats.js index d2fd6eac..6fb0fbba 100644 --- a/rel/overlay/share/www/script/test/stats.js +++ b/rel/overlay/share/www/script/test/stats.js @@ -30,7 +30,7 @@ couchTests.stats = function(debug) { _id:"_design/test", // turn off couch.js id escaping? language: "javascript", views: { - all_docs: {map: "function(doc) {emit(doc.integer, null);}"}, + all_docs: {map: "function(doc) {emit(doc.integer, null);}"} } }; db.save(designDoc); @@ -163,12 +163,12 @@ couchTests.stats = function(debug) { CouchDB.request("POST", "/test_suite_db", { headers: {"Content-Type": "application/json"}, body: '{"a": "1"}' - }) + }); }, test: function(before, after) { TEquals(before+1, after, "POST'ing new docs increments doc writes."); } - }) + }); runTest("couchdb", "database_writes", { setup: function(db) {db.save({"_id": "test"});}, @@ -247,7 +247,7 @@ couchTests.stats = function(debug) { }); runTest("httpd", "temporary_view_reads", { - run: function(db) {db.query(function(doc) {emit(doc._id)})}, + run: function(db) { db.query(function(doc) { emit(doc._id); }); }, test: function(before, after) { TEquals(before+1, after, "Temporary views have their own counter."); } @@ -261,7 +261,7 @@ couchTests.stats = function(debug) { }); runTest("httpd", "view_reads", { - run: function(db) {db.query(function(doc) {emit(doc._id)});}, + run: function(db) { db.query(function(doc) { emit(doc._id); }); }, test: function(before, after) { TEquals(before, after, "Temporary views don't affect permanent views."); } diff --git a/rel/overlay/share/www/script/test/users_db.js b/rel/overlay/share/www/script/test/users_db.js index 667ff3c1..1e13e5d7 100644 --- a/rel/overlay/share/www/script/test/users_db.js +++ b/rel/overlay/share/www/script/test/users_db.js @@ -90,6 +90,27 @@ couchTests.users_db = function(debug) { T(s.name == null); T(s.roles.indexOf("_admin") !== -1); T(usersDb.deleteDoc(jchrisWithConflict).ok); + + // you can't change doc from type "user" + jchrisUserDoc = usersDb.open(jchrisUserDoc._id); + jchrisUserDoc.type = "not user"; + try { + usersDb.save(jchrisUserDoc); + T(false && "should only allow us to save doc when type == 'user'"); + } catch(e) { + T(e.reason == "doc.type must be user"); + } + jchrisUserDoc.type = "user"; + + // "roles" must be an array + jchrisUserDoc.roles = "not an array"; + try { + usersDb.save(jchrisUserDoc); + T(false && "should only allow us to save doc when roles is an array"); + } catch(e) { + T(e.reason == "doc.roles must be an array"); + } + jchrisUserDoc.roles = []; }; usersDb.deleteDb(); @@ -100,4 +121,4 @@ couchTests.users_db = function(debug) { ); usersDb.deleteDb(); // cleanup -}
\ No newline at end of file +} diff --git a/rel/overlay/share/www/script/test/view_errors.js b/rel/overlay/share/www/script/test/view_errors.js index a211c061..c05000b7 100644 --- a/rel/overlay/share/www/script/test/view_errors.js +++ b/rel/overlay/share/www/script/test/view_errors.js @@ -74,9 +74,6 @@ couchTests.view_errors = function(debug) { T(e.error == "query_parse_error"); } - // reduce=false on map views doesn't work, so group=true will - // never throw for temp reduce views. - var designDoc = { _id:"_design/test", language: "javascript", @@ -104,6 +101,15 @@ couchTests.view_errors = function(debug) { db.view("test/no_reduce", {group: true}); T(0 == 1); } catch(e) { + T(db.last_req.status == 400); + T(e.error == "query_parse_error"); + } + + try { + db.view("test/no_reduce", {group_level: 1}); + T(0 == 1); + } catch(e) { + T(db.last_req.status == 400); T(e.error == "query_parse_error"); } @@ -115,10 +121,23 @@ couchTests.view_errors = function(debug) { T(e.error == "query_parse_error"); } + db.view("test/no_reduce", {reduce: false}); + TEquals(200, db.last_req.status, "reduce=false for map views (without" + + " group or group_level) is allowed"); + try { db.view("test/with_reduce", {group: true, reduce: false}); T(0 == 1); } catch(e) { + T(db.last_req.status == 400); + T(e.error == "query_parse_error"); + } + + try { + db.view("test/with_reduce", {group_level: 1, reduce: false}); + T(0 == 1); + } catch(e) { + T(db.last_req.status == 400); T(e.error == "query_parse_error"); } diff --git a/rel/overlay/share/www/script/test/view_multi_key_design.js b/rel/overlay/share/www/script/test/view_multi_key_design.js index 5a2f645d..c39e73d9 100644 --- a/rel/overlay/share/www/script/test/view_multi_key_design.js +++ b/rel/overlay/share/www/script/test/view_multi_key_design.js @@ -34,11 +34,11 @@ couchTests.view_multi_key_design = function(debug) { reduce:"function (keys, values) { return sum(values); };" } } - } + }; T(db.save(designDoc).ok); // Test that missing keys work too - var keys = [101,30,15,37,50] + var keys = [101,30,15,37,50]; var reduce = db.view("test/summate",{group:true},keys).rows; T(reduce.length == keys.length-1); // 101 is missing for(var i=0; i<reduce.length; i++) { @@ -81,7 +81,7 @@ couchTests.view_multi_key_design = function(debug) { } // Test that a map & reduce containing func support keys when reduce=false - resp = db.view("test/summate", {reduce: false}, keys); + var resp = db.view("test/summate", {reduce: false}, keys); T(resp.rows.length == 5); // Check that limiting by startkey_docid and endkey_docid get applied diff --git a/rel/overlay/share/www/script/test/view_sandboxing.js b/rel/overlay/share/www/script/test/view_sandboxing.js index 9f893b28..02951d9f 100644 --- a/rel/overlay/share/www/script/test/view_sandboxing.js +++ b/rel/overlay/share/www/script/test/view_sandboxing.js @@ -42,11 +42,99 @@ couchTests.view_sandboxing = function(debug) { // make sure that a view cannot access the map_funs array defined used by // the view server - var results = db.query(function(doc) { map_funs.push(1); emit(null, doc) }); + var results = db.query(function(doc) { map_funs.push(1); emit(null, doc); }); T(results.total_rows == 0); // make sure that a view cannot access the map_results array defined used by // the view server - var results = db.query(function(doc) { map_results.push(1); emit(null, doc) }); + var results = db.query(function(doc) { map_results.push(1); emit(null, doc); }); T(results.total_rows == 0); + + // test for COUCHDB-925 + // altering 'doc' variable in map function affects other map functions + var ddoc = { + _id: "_design/foobar", + language: "javascript", + views: { + view1: { + map: + (function(doc) { + if (doc.values) { + doc.values = [666]; + } + if (doc.tags) { + doc.tags.push("qwerty"); + } + if (doc.tokens) { + doc.tokens["c"] = 3; + } + }).toString() + }, + view2: { + map: + (function(doc) { + if (doc.values) { + emit(doc._id, doc.values); + } + if (doc.tags) { + emit(doc._id, doc.tags); + } + if (doc.tokens) { + emit(doc._id, doc.tokens); + } + }).toString() + } + } + }; + var doc1 = { + _id: "doc1", + values: [1, 2, 3] + }; + var doc2 = { + _id: "doc2", + tags: ["foo", "bar"], + tokens: {a: 1, b: 2} + }; + + db.deleteDb(); + db.createDb(); + T(db.save(ddoc).ok); + T(db.save(doc1).ok); + T(db.save(doc2).ok); + + var view1Results = db.view( + "foobar/view1", {bypass_cache: Math.round(Math.random() * 1000)}); + var view2Results = db.view( + "foobar/view2", {bypass_cache: Math.round(Math.random() * 1000)}); + + TEquals(0, view1Results.rows.length, "view1 has 0 rows"); + TEquals(3, view2Results.rows.length, "view2 has 3 rows"); + + TEquals(doc1._id, view2Results.rows[0].key); + TEquals(doc2._id, view2Results.rows[1].key); + TEquals(doc2._id, view2Results.rows[2].key); + + // https://bugzilla.mozilla.org/show_bug.cgi?id=449657 + TEquals(3, view2Results.rows[0].value.length, + "Warning: installed SpiderMonkey version doesn't allow sealing of arrays"); + if (view2Results.rows[0].value.length === 3) { + TEquals(1, view2Results.rows[0].value[0]); + TEquals(2, view2Results.rows[0].value[1]); + TEquals(3, view2Results.rows[0].value[2]); + } + + TEquals(1, view2Results.rows[1].value["a"]); + TEquals(2, view2Results.rows[1].value["b"]); + TEquals('undefined', typeof view2Results.rows[1].value["c"], + "doc2.tokens object was not sealed"); + + TEquals(2, view2Results.rows[2].value.length, + "Warning: installed SpiderMonkey version doesn't allow sealing of arrays"); + if (view2Results.rows[2].value.length === 2) { + TEquals("foo", view2Results.rows[2].value[0]); + TEquals("bar", view2Results.rows[2].value[1]); + } + + // cleanup + db.deleteDb(); }; diff --git a/rel/overlay/share/www/script/test/view_update_seq.js b/rel/overlay/share/www/script/test/view_update_seq.js index e6be3f70..9757caa1 100644 --- a/rel/overlay/share/www/script/test/view_update_seq.js +++ b/rel/overlay/share/www/script/test/view_update_seq.js @@ -18,7 +18,7 @@ couchTests.view_update_seq = function(debug) { T(db.info().update_seq == 0); - resp = db.allDocs({update_seq:true}); + var resp = db.allDocs({update_seq:true}); T(resp.rows.length == 0); T(resp.update_seq == 0); @@ -35,7 +35,7 @@ couchTests.view_update_seq = function(debug) { reduce:"function (keys, values) { return sum(values); };" } } - } + }; T(db.save(designDoc).ok); T(db.info().update_seq == 1); diff --git a/rel/overlay/share/www/session.html b/rel/overlay/share/www/session.html index 581640b0..0ebd943d 100644 --- a/rel/overlay/share/www/session.html +++ b/rel/overlay/share/www/session.html @@ -36,7 +36,7 @@ specific language governing permissions and limitations under the License. } m = qp.match(/reason=(.*)/); if (m) { - reason = decodeURIComponent(m[1]); + reason = $.futon.escape(decodeURIComponent(m[1])); } }); if (reason) { diff --git a/share/www/script/test/attachment_conflicts.js b/share/www/script/test/attachment_conflicts.js new file mode 100644 index 00000000..c400277e --- /dev/null +++ b/share/www/script/test/attachment_conflicts.js @@ -0,0 +1,56 @@ +// 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. + +// Do some edit conflict detection tests for attachments. +couchTests.attachment_conflicts = function(debug) { + + var dbA = new CouchDB("test_suite_db_a", {"X-Couch-Full-Commit":"false"}); + var dbB = new CouchDB("test_suite_db_b", {"X-Couch-Full-Commit":"false"}); + dbA.deleteDb(); + dbA.createDb(); + dbB.deleteDb(); + dbB.createDb(); + + if (debug) debugger; + + T(dbA.save({"_id":"doc", "foo":"bar"}).ok); + + // create conflict + T(CouchDB.replicate("test_suite_db_a", "test_suite_db_b").ok); + + var doc = dbA.open("doc"); + var rev11 = doc._rev; + T(dbA.save({"_id":"doc", "foo":"bar2","_rev":rev11}).ok); + + doc = dbB.open("doc"); + var rev12 = doc._rev; + T(dbB.save({"_id":"doc", "foo":"bar3","_rev":rev12}).ok); + + T(CouchDB.replicate("test_suite_db_a", "test_suite_db_b").ok); + + // the attachment + var bin_data = "JHAPDO*AU£PN ){(3u[d 93DQ9¡€])} ææøo'∂ƒæ≤çæππ•¥∫¶®#†π¶®¥π€ª®˙π8np"; + + doc = dbB.open("doc"); + var rev13 = doc._rev; + + // test that we can can attach to conflicting documents + var xhr = CouchDB.request("PUT", "/test_suite_db_b/doc/attachment.txt", { + headers: { + "Content-Type": "text/plain;charset=utf-8", + "If-Match": rev13 + }, + body: bin_data + }); + T(xhr.status == 201); + +}; |