diff options
Diffstat (limited to 'src')
| -rw-r--r-- | src/couchdb/couch_db.erl | 179 | ||||
| -rw-r--r-- | src/couchdb/couch_db.hrl | 4 | ||||
| -rw-r--r-- | src/couchdb/couch_db_updater.erl | 58 | ||||
| -rw-r--r-- | src/couchdb/couch_doc.erl | 42 | ||||
| -rw-r--r-- | src/couchdb/couch_erl_driver.c | 8 | ||||
| -rw-r--r-- | src/couchdb/couch_httpd.erl | 53 | ||||
| -rw-r--r-- | src/couchdb/couch_httpd_db.erl | 8 | ||||
| -rw-r--r-- | src/couchdb/couch_key_tree.erl | 12 | ||||
| -rw-r--r-- | src/couchdb/couch_query_servers.erl | 25 | ||||
| -rw-r--r-- | src/couchdb/couch_rep.erl | 12 | ||||
| -rw-r--r-- | src/couchdb/couch_server.erl | 23 | 
11 files changed, 291 insertions, 133 deletions
| diff --git a/src/couchdb/couch_db.erl b/src/couchdb/couch_db.erl index e1b36f42..6441e2e1 100644 --- a/src/couchdb/couch_db.erl +++ b/src/couchdb/couch_db.erl @@ -14,14 +14,14 @@  -behaviour(gen_server).  -export([open/2,close/1,create/2,start_compact/1,get_db_info/1]). --export([open_ref_counted/2,num_refs/1,monitor/1]). --export([save_docs/3,update_doc/3,update_docs/2,update_docs/3,delete_doc/3]). +-export([open_ref_counted/3,num_refs/1,monitor/1]). +-export([update_doc/3,update_docs/4,update_docs/2,update_docs/3,delete_doc/3]).  -export([get_doc_info/2,open_doc/2,open_doc/3,open_doc_revs/4]). --export([get_missing_revs/2,name/1]). +-export([get_missing_revs/2,name/1,doc_to_tree/1]).  -export([enum_docs/4,enum_docs/5,enum_docs_since/4,enum_docs_since/5]).  -export([enum_docs_since_reduce_to_count/1,enum_docs_reduce_to_count/1]).  -export([increment_update_seq/1,get_purge_seq/1,purge_docs/2,get_last_purged/1]). --export([start_link/3]). +-export([start_link/3,make_doc/2]).  -export([init/1,terminate/2,handle_call/3,handle_cast/2,code_change/3,handle_info/2]). @@ -33,19 +33,7 @@ start_link(DbName, Filepath, Options) ->      {ok, Fd} ->          StartResult = gen_server:start_link(couch_db, {DbName, Filepath, Fd, Options}, []),          unlink(Fd), -        case StartResult of -        {ok, _} -> -            % We successfully opened the db, delete old storage files if around -            case file:delete(Filepath ++ ".old") of -            ok -> -                ?LOG_INFO("Deleted old storage file ~s~s", [Filepath, ".old"]); -            {error, enoent} -> -                ok  % normal result -            end, -            StartResult; -        Error -> -            Error -        end; +        StartResult;      Else ->          Else      end. @@ -79,8 +67,9 @@ open(DbName, Options) ->  close(#db{fd=Fd}) ->      couch_file:drop_ref(Fd). -open_ref_counted(MainPid, OpeningPid) -> -    gen_server:call(MainPid, {open_ref_counted_instance, OpeningPid}). +open_ref_counted(MainPid, OpeningPid, UserCred) -> +    {ok, Db} = gen_server:call(MainPid, {open_ref_counted_instance, OpeningPid}), +    {ok, Db#db{user_ctx=UserCred}}.  num_refs(MainPid) ->      gen_server:call(MainPid, num_refs). @@ -213,39 +202,92 @@ group_alike_docs([Doc|Rest], [Bucket|RestBuckets]) ->          % add to new bucket         group_alike_docs(Rest, [[Doc]|[Bucket|RestBuckets]])      end. -     -prepare_doc_for_new_edit(Db, #doc{id=Id,revs=[NewRev|PrevRevs]}=Doc, OldFullDocInfo, LeafRevsDict) -> +validate_doc_update(#db{validate_doc_funs=[]}, Doc, _GetDiskDocFun) -> +    Doc; +validate_doc_update(_Db, #doc{id= <<"_design/",_/binary>>}=Doc, _GetDiskDocFun) -> +    Doc; +validate_doc_update(_Db, #doc{id= <<"_local/",_/binary>>}=Doc, _GetDiskDocFun) -> +    Doc; +validate_doc_update(#db{name=DbName,user_ctx={CtxProps}}=Db, Doc, GetDiskDocFun) -> +    DiskDoc = GetDiskDocFun(), +    [case Fun(Doc, DiskDoc, {[{<<"db">>, DbName} | CtxProps]}) of +        ok -> ok; +        Error -> throw(Error) +    end || Fun <- Db#db.validate_doc_funs], +    Doc. + + +prep_and_validate_new_edit(Db, #doc{id=Id,revs=[NewRev|PrevRevs]}=Doc, +        OldFullDocInfo, LeafRevsDict) ->      case PrevRevs of      [PrevRev|_] ->          case dict:find(PrevRev, LeafRevsDict) of          {ok, {Deleted, Sp, DiskRevs}} -> -            case couch_doc:has_stubs(Doc) of +            Doc2 = Doc#doc{revs=[NewRev|DiskRevs]}, +            case couch_doc:has_stubs(Doc2) of              true ->                  DiskDoc = make_doc(Db, Id, Deleted, Sp, DiskRevs), -                Doc2 = couch_doc:merge_stubs(Doc, DiskDoc), -                Doc2#doc{revs=[NewRev|DiskRevs]}; +                Doc3 = couch_doc:merge_stubs(Doc2, DiskDoc), +                validate_doc_update(Db, Doc3, fun() -> DiskDoc end);              false -> -                Doc#doc{revs=[NewRev|DiskRevs]} +                LoadDiskDoc = fun() -> make_doc(Db,Id,Deleted,Sp,DiskRevs) end, +                validate_doc_update(Db, Doc2, LoadDiskDoc)              end;          error ->              throw(conflict)          end;      [] -> -        % new doc, and we have existing revs.  -        OldDocInfo = couch_doc:to_doc_info(OldFullDocInfo), -        if OldDocInfo#doc_info.deleted -> -            % existing doc is a deleton -            % allow this new doc to be a later revision. -            {_Deleted, _Sp, Revs} = dict:fetch(OldDocInfo#doc_info.rev, LeafRevsDict), -            Doc#doc{revs=[NewRev|Revs]}; +        % new doc, and we have existing revs. +        if OldFullDocInfo#full_doc_info.deleted -> +            % existing docs are deletions +            validate_doc_update(Db, Doc, nil);          true ->              throw(conflict)          end      end.  update_docs(#db{update_pid=UpdatePid}=Db, Docs, Options) -> -    % go ahead and generate the new revision ids for the documents. +    update_docs(#db{update_pid=UpdatePid}=Db, Docs, Options, true). + +update_docs(Db, Docs, Options, false) -> +    DocBuckets = group_alike_docs(Docs), +    Ids = [Id || [#doc{id=Id}|_] <- DocBuckets], +     +    ExistingDocs = get_full_doc_infos(Db, Ids), +     +    DocBuckets2 = lists:zipwith( +        fun(Bucket, not_found) -> +            [validate_doc_update(Db, Doc, fun()-> nil end) || Doc <- Bucket]; +        (Bucket, {ok, #full_doc_info{rev_tree=OldRevTree}}) -> +            NewTree = lists:foldl( +                fun(Doc, RevTreeAcc) -> +                    couch_key_tree:merge(RevTreeAcc, doc_to_tree(Doc)) +                end, +                OldRevTree, Bucket), +            Leafs = couch_key_tree:get_all_leafs_full(NewTree), +            LeafRevsFullDict = dict:from_list( [{Rev, FullPath} || [{Rev, _}|_]=FullPath <- Leafs]), +            lists:flatmap( +                fun(#doc{revs=[Rev|_]}=Doc) -> +                    case dict:find(Rev, LeafRevsFullDict) of +                    {ok, [{Rev, #doc{id=Id}}|_]=Path} -> +                        % our unflushed doc is a leaf node. Go back on the path  +                        % to find the previous rev that's on disk. +                        LoadPrevRev = fun() -> +                            make_first_doc_on_disk(Db, Id, Path) +                        end, +                        [validate_doc_update(Db, Doc, LoadPrevRev)]; +                    _ -> +                        % this doc isn't a leaf or is already exists in the tree. ignore +                        [] +                    end +                end, Bucket) +        end, +        DocBuckets, ExistingDocs), +    write_and_commit(Db, DocBuckets2, Options); +     +update_docs(Db, Docs, Options, true) -> +        % go ahead and generate the new revision ids for the documents.      Docs2 = lists:map(          fun(#doc{id=Id,revs=Revs}=Doc) ->              case Id of @@ -256,7 +298,6 @@ update_docs(#db{update_pid=UpdatePid}=Db, Docs, Options) ->                  Doc#doc{revs=[list_to_binary(integer_to_list(couch_util:rand32())) | Revs]}              end          end, Docs), -    NewRevs = [NewRev || #doc{revs=[NewRev|_]} <- Docs2],      DocBuckets = group_alike_docs(Docs2),      Ids = [Id || [#doc{id=Id}|_] <- DocBuckets], @@ -266,39 +307,51 @@ update_docs(#db{update_pid=UpdatePid}=Db, Docs, Options) ->      DocBuckets2 = lists:zipwith(          fun(Bucket, not_found) -> -            % no existing revs, make sure no old revision is specified. +            % no existing revs on disk, make sure no old revs specified.              [throw(conflict) || #doc{revs=[_NewRev, _OldRev | _]} <- Bucket], -            Bucket; +            [validate_doc_update(Db, Doc, fun()-> nil end) || Doc <- Bucket];          (Bucket, {ok, #full_doc_info{rev_tree=OldRevTree}=OldFullDocInfo}) ->              Leafs = couch_key_tree:get_all_leafs(OldRevTree),              LeafRevsDict = dict:from_list([{Rev, {Deleted, Sp, Revs}} || {Rev, {Deleted, Sp}, Revs} <- Leafs]), -            [prepare_doc_for_new_edit(Db, Doc, OldFullDocInfo, LeafRevsDict) || Doc <- Bucket] +            [prep_and_validate_new_edit(Db, Doc, OldFullDocInfo, LeafRevsDict) || Doc <- Bucket]          end,          DocBuckets, ExistingDocs), -    % flush unwritten binaries to disk. -    DocBuckets3 = [[doc_flush_binaries(Doc, Db#db.fd) || Doc <- Bucket] || Bucket <- DocBuckets2], +    ok = write_and_commit(Db, DocBuckets2, [new_edits | Options]), +    {ok, [NewRev ||#doc{revs=[NewRev|_]} <- Docs2]}. + + +% Returns the first available document on disk. Input list is a full rev path +% for the doc. +make_first_doc_on_disk(_Db, _Id, []) -> +    nil; +make_first_doc_on_disk(Db, Id, [{_Rev, ?REV_MISSING}|RestPath]) -> +    make_first_doc_on_disk(Db, Id, RestPath); +make_first_doc_on_disk(Db, Id, [{_Rev, {IsDel, Sp}} |_]=DocPath) -> +    Revs = [Rev || {Rev, _} <- DocPath], +    make_doc(Db, Id, IsDel, Sp, Revs). -    case gen_server:call(UpdatePid, {update_docs, DocBuckets3, [new_edits | Options]}, infinity) of -    ok -> {ok, NewRevs}; + +write_and_commit(#db{update_pid=UpdatePid}=Db, DocBuckets, Options) -> + +    % flush unwritten binaries to disk. +    DocBuckets2 = [[doc_flush_binaries(Doc, Db#db.fd) || Doc <- Bucket] || Bucket <- DocBuckets], +    case gen_server:call(UpdatePid, {update_docs, DocBuckets2, Options}, infinity) of +    ok -> ok;      retry -> -        {ok, Db2} = open_ref_counted(Db#db.main_pid, self()), -        DocBuckets4 = [[doc_flush_binaries(Doc, Db2#db.fd) || Doc <- Bucket] || Bucket <- DocBuckets3], +        % This can happen if the db file we wrote to was swapped out by +        % compaction. Retry writing to the current file +        {ok, Db2} = open_ref_counted(Db#db.main_pid, self(), {[]}), +        DocBuckets3 = [[doc_flush_binaries(Doc, Db2#db.fd) || Doc <- Bucket] || Bucket <- DocBuckets],          % We only retry once          close(Db2), -        case gen_server:call(UpdatePid, {update_docs, DocBuckets4, [new_edits | Options]}, infinity) of -        ok -> {ok, NewRevs}; +        case gen_server:call(UpdatePid, {update_docs, DocBuckets3, Options}, infinity) of +        ok -> ok;          Else -> throw(Else)          end;      Else->          throw(Else)      end. -save_docs(#db{update_pid=UpdatePid, fd=Fd}, Docs, Options) -> -    % flush unwritten binaries to disk. -    DocBuckets = group_alike_docs(Docs), -    DocBuckets2 = [[doc_flush_binaries(Doc, Fd) || Doc <- Bucket] || Bucket <- DocBuckets], -    ok = gen_server:call(UpdatePid, {update_docs, DocBuckets2, Options}, infinity). -  doc_flush_binaries(Doc, Fd) ->      % calc size of binaries to write out @@ -509,14 +562,30 @@ doc_meta_info(DocInfo, RevTree, Options) ->          end      end. -make_doc(Db, Id, Deleted, SummaryPointer, RevisionPath) -> + +doc_to_tree(Doc) -> +    doc_to_tree(Doc, lists:reverse(Doc#doc.revs)). + +doc_to_tree(Doc, [RevId]) -> +    [{RevId, Doc, []}]; +doc_to_tree(Doc, [RevId | Rest]) -> +    [{RevId, ?REV_MISSING, doc_to_tree(Doc, Rest)}]. + +make_doc(Db, FullDocInfo) -> +    {#doc_info{id=Id,deleted=Deleted,summary_pointer=Sp}, RevPath} +            = couch_doc:to_doc_info_path(FullDocInfo), +    make_doc(Db, Id, Deleted, Sp, RevPath). +     +make_doc(#db{fd=Fd}=Db, Id, Deleted, BodySp, RevisionPath) ->      {BodyData, BinValues} = -    case SummaryPointer of +    case BodySp of      nil ->          {[], []};      _ -> -        {ok, {BodyData0, BinValues0}} = couch_stream:read_term(Db#db.summary_stream, SummaryPointer), -        {BodyData0, [{Name, {Type, {Db#db.fd, Sp, Len}}} || {Name, {Type, Sp, Len}} <- BinValues0]}    +        {ok, {BodyData0, BinValues0}} = +            couch_stream:read_term( Db#db.summary_stream, BodySp), +        {BodyData0, +            [{Name,{Type,{Fd,Sp,Len}}} || {Name,{Type,Sp,Len}} <- BinValues0]}      end,      #doc{          id = Id, diff --git a/src/couchdb/couch_db.hrl b/src/couchdb/couch_db.hrl index d1825d9e..51f9b311 100644 --- a/src/couchdb/couch_db.hrl +++ b/src/couchdb/couch_db.hrl @@ -111,7 +111,9 @@      local_docs_btree,      update_seq,      name, -    filepath +    filepath, +    validate_doc_funs=[], +    user_ctx={[]}      }). diff --git a/src/couchdb/couch_db_updater.erl b/src/couchdb/couch_db_updater.erl index 6982fb01..b3df910f 100644 --- a/src/couchdb/couch_db_updater.erl +++ b/src/couchdb/couch_db_updater.erl @@ -28,13 +28,13 @@ init({MainPid, DbName, Filepath, Fd, Options}) ->          Header =  #db_header{},          ok = couch_file:write_header(Fd, ?HEADER_SIG, Header),          % delete any old compaction files that might be hanging around -        file:delete(Filepath ++ ".compact"), -        file:delete(Filepath ++ ".old"); +        file:delete(Filepath ++ ".compact");      false ->          {ok, Header} = couch_file:read_header(Fd, ?HEADER_SIG)      end,      Db = init_db(DbName, Filepath, Fd, Header), +    Db2 = refresh_validate_doc_funs(Db),      {ok, Db#db{main_pid=MainPid}}.  terminate(_Reason, Db) -> @@ -158,7 +158,6 @@ handle_cast({compact_done, CompactFilepath}, #db{filepath=Filepath}=Db) ->          couch_stream:close(Db#db.summary_stream),          couch_file:close_maybe(Db#db.fd), -        file:delete(Filepath ++ ".old"),          ok = gen_server:call(Db#db.main_pid, {db_updated, NewDb2}),          ?LOG_INFO("Compaction for db \"~s\" completed.", [Db#db.name]), @@ -254,21 +253,34 @@ init_db(DbName, Filepath, Fd, Header) ->          local_docs_btree = LocalDocsBtree,          update_seq = Header#db_header.update_seq,          name = DbName, -        filepath=Filepath }. +        filepath=Filepath}. +  close_db(#db{fd=Fd,summary_stream=Ss}) ->      couch_file:close(Fd),      couch_stream:close(Ss). -% rev tree functions - -doc_to_tree(Doc) -> -    doc_to_tree(Doc, lists:reverse(Doc#doc.revs)). +refresh_validate_doc_funs(Db) -> +    {ok, DesignDocs} = get_design_docs(Db), +    ProcessDocFuns = lists:flatmap( +        fun(DesignDoc) -> +            case couch_doc:get_validate_doc_fun(DesignDoc) of +            nil -> []; +            Fun -> [Fun] +            end +        end, DesignDocs), +    Db#db{validate_doc_funs=ProcessDocFuns}. + +get_design_docs(#db{fulldocinfo_by_id_btree=Btree}=Db) -> +    couch_btree:foldl(Btree, <<"_design/">>, +        fun(#full_doc_info{id= <<"_design/",_/binary>>}=FullDocInfo, _Reds, AccDocs) -> +            {ok, [couch_db:make_doc(Db, FullDocInfo) | AccDocs]}; +        (_, _Reds, AccDocs) -> +            {stop, AccDocs} +        end, +        []). -doc_to_tree(Doc, [RevId]) -> -    [{RevId, Doc, []}]; -doc_to_tree(Doc, [RevId | Rest]) -> -    [{RevId, ?REV_MISSING, doc_to_tree(Doc, Rest)}]. +% rev tree functions  flush_trees(_Db, [], AccFlushedTrees) ->      {ok, lists:reverse(AccFlushedTrees)}; @@ -311,7 +323,7 @@ merge_rev_trees(NoConflicts, [NewDocs|RestDocsList],      #full_doc_info{id=Id,rev_tree=OldTree}=OldDocInfo,      UpdatesRevTree = lists:foldl(          fun(NewDoc, AccTree) -> -            couch_key_tree:merge(AccTree, doc_to_tree(NewDoc)) +            couch_key_tree:merge(AccTree, couch_db:doc_to_tree(NewDoc))          end,          [], NewDocs),      NewRevTree = couch_key_tree:merge(OldTree, UpdatesRevTree), @@ -323,7 +335,14 @@ merge_rev_trees(NoConflicts, [NewDocs|RestDocsList],              OldConflicts = couch_key_tree:count_leafs(OldTree),              NewConflicts = couch_key_tree:count_leafs(NewRevTree),              if NewConflicts > OldConflicts -> -                throw(conflict); +                % if all the old docs are deletions, allow this new conflict +                case [1 || {_Rev,{IsDel,_Sp}} <-  +                    couch_key_tree:get_all_leafs(OldTree), IsDel==false] of +                [] -> +                    ok; +                _ -> +                    throw(conflict) +                end;              true -> ok              end;          true -> ok @@ -397,12 +416,19 @@ update_docs_int(Db, DocsList, Options) ->          fulldocinfo_by_id_btree = DocInfoByIdBTree2,          docinfo_by_seq_btree = DocInfoBySeqBTree2,          update_seq = NewSeq}, +     +    case [1 || <<"_design/",_/binary>> <- Ids] of +    [] -> +        Db4 = Db3; +    _ -> +        Db4 = refresh_validate_doc_funs(Db3) +    end,      case lists:member(delay_commit, Options) of      true -> -        {ok, Db3}; +        {ok, Db4};      false -> -        {ok, commit_data(Db3)} +        {ok, commit_data(Db4)}      end.  update_local_docs(#db{local_docs_btree=Btree}=Db, Docs) -> diff --git a/src/couchdb/couch_doc.erl b/src/couchdb/couch_doc.erl index 3efb8bd7..56d559ea 100644 --- a/src/couchdb/couch_doc.erl +++ b/src/couchdb/couch_doc.erl @@ -12,8 +12,8 @@  -module(couch_doc). --export([get_view_functions/1, is_special_doc/1,to_doc_info/1]). --export([bin_foldl/3,bin_size/1,bin_to_binary/1]). +-export([to_doc_info/1,to_doc_info_path/1]). +-export([bin_foldl/3,bin_size/1,bin_to_binary/1,get_validate_doc_fun/1]).  -export([from_json_obj/1,to_json_obj/2,has_stubs/1, merge_stubs/2]).  -include("couch_db.hrl"). @@ -130,8 +130,11 @@ from_json_obj({Props}) ->          attachments = Bins          }. +to_doc_info(FullDocInfo) -> +    {DocInfo, _Path} = to_doc_info_path(FullDocInfo), +    DocInfo. -to_doc_info(#full_doc_info{id=Id,update_seq=Seq,rev_tree=Tree}) -> +to_doc_info_path(#full_doc_info{id=Id,update_seq=Seq,rev_tree=Tree}) ->      LeafRevs = couch_key_tree:get_all_leafs(Tree),      SortedLeafRevs =      lists:sort(fun({RevIdA, {IsDeletedA, _}, PathA}, {RevIdB, {IsDeletedB, _}, PathB}) -> @@ -142,7 +145,7 @@ to_doc_info(#full_doc_info{id=Id,update_seq=Seq,rev_tree=Tree}) ->          end,          LeafRevs), -    [{RevId, {IsDeleted, SummaryPointer}, _Path} | Rest] = SortedLeafRevs, +    [{RevId, {IsDeleted, SummaryPointer}, Path} | Rest] = SortedLeafRevs,      {ConflictRevTuples, DeletedConflictRevTuples} =          lists:splitwith(fun({_ConflictRevId, {IsDeleted1, _Sp}, _}) -> @@ -151,22 +154,15 @@ to_doc_info(#full_doc_info{id=Id,update_seq=Seq,rev_tree=Tree}) ->      ConflictRevs = [RevId1  || {RevId1, _, _} <- ConflictRevTuples],      DeletedConflictRevs = [RevId2   || {RevId2, _, _} <- DeletedConflictRevTuples], -    #doc_info{ +    DocInfo = #doc_info{          id=Id,          update_seq=Seq,          rev = RevId,          summary_pointer = SummaryPointer,          conflict_revs = ConflictRevs,          deleted_conflict_revs = DeletedConflictRevs, -        deleted = IsDeleted -        }. - -is_special_doc(?DESIGN_DOC_PREFIX ++ _ ) -> -    true; -is_special_doc(#doc{id=Id}) -> -    is_special_doc(Id); -is_special_doc(_) -> -    false. +        deleted = IsDeleted}, +    {DocInfo, Path}.  bin_foldl(Bin, Fun, Acc) when is_binary(Bin) ->      case Fun(Bin, Acc) of @@ -188,12 +184,18 @@ bin_to_binary({Fd, Sp, Len}) ->      {ok, Bin, _Sp2} = couch_stream:read(Fd, Sp, Len),      Bin. -get_view_functions(#doc{body={Fields}}) -> -    Lang = proplists:get_value(<<"language">>, Fields, <<"javascript">>), -    {Views} = proplists:get_value(<<"views">>, Fields, {[]}), -    {Lang, [{ViewName, Value} || {ViewName, Value} <- Views, is_list(Value)]}; -get_view_functions(_Doc) -> -    none. +get_validate_doc_fun(#doc{body={Props}}) -> +    Lang = proplists:get_value(<<"language">>, Props, <<"javascript">>), +    case proplists:get_value(<<"validate_doc_update">>, Props) of +    undefined -> +        nil; +    FunSrc -> +        fun(EditDoc, DiskDoc, Ctx) -> +            couch_query_servers:validate_doc_update( +                    Lang, FunSrc, EditDoc, DiskDoc, Ctx) +        end +    end. +          has_stubs(#doc{attachments=Bins}) ->      has_stubs(Bins); diff --git a/src/couchdb/couch_erl_driver.c b/src/couchdb/couch_erl_driver.c index e3cbf4ab..18fabc2b 100644 --- a/src/couchdb/couch_erl_driver.c +++ b/src/couchdb/couch_erl_driver.c @@ -14,8 +14,8 @@ specific language governing permissions and limitations under the License.  */  // This file is the C port driver for Erlang. It provides a low overhead -// means of calling into C code, however unlike the Fabric engine, coding -// errors in this module can crash the entire Erlang server. +// means of calling into C code, however coding errors in this module can +// crash the entire Erlang server.  #ifdef DARWIN  #define U_HIDE_DRAFT_API 1 @@ -56,10 +56,8 @@ static ErlDrvData couch_drv_start(ErlDrvPort port, char *buff)          return ERL_DRV_ERROR_GENERAL;      pData->port = port; -    pData->coll = NULL; -    pData->collNoCase = NULL; +          pData->coll = ucol_open("", &status); -      if (U_FAILURE(status)) {          couch_drv_stop((ErlDrvData)pData);          return ERL_DRV_ERROR_GENERAL; diff --git a/src/couchdb/couch_httpd.erl b/src/couchdb/couch_httpd.erl index 2be90893..c58255e3 100644 --- a/src/couchdb/couch_httpd.erl +++ b/src/couchdb/couch_httpd.erl @@ -16,7 +16,7 @@  -export([start_link/0, stop/0, handle_request/3]).  -export([header_value/2,header_value/3,qs_value/2,qs_value/3,qs/1,path/1]). --export([check_is_admin/1,unquote/1]). +-export([check_is_admin/1,unquote/1,creds/1]).  -export([parse_form/1,json_body/1,body/1,doc_etag/1]).  -export([primary_header_value/2,partition/1,serve_file/3]).  -export([start_chunked_response/3,send_chunk/2]). @@ -197,14 +197,37 @@ json_body(#httpd{mochi_req=MochiReq}) ->  doc_etag(#doc{revs=[DiskRev|_]}) ->       "\"" ++ binary_to_list(DiskRev) ++ "\"". -check_is_admin(Req) -> -    IsNamedAdmin = + +% user credentials +creds(Req) -> +    case username_pw(Req) of +    {User, _Pw} -> +        {[{<<"name">>, ?l2b(User)}]}; +    nil -> +        {[]} +    end. + +username_pw(Req) ->      case header_value(Req, "Authorization") of      "Basic " ++ Base64Value -> -        [User, Pass] = -            string:tokens(?b2l(couch_util:decodeBase64(Base64Value)),":"), -        couch_server:is_admin(User, Pass); +        case string:tokens(?b2l(couch_util:decodeBase64(Base64Value)),":") of +        [User, Pass] -> +            {User, Pass}; +        [User] -> +            {User, <<"">>}; +        _ -> +            nil +        end;      _ -> +        nil +    end. + +check_is_admin(Req) -> +    IsNamedAdmin = +    case username_pw(Req) of +    {User, Pass} -> +        couch_server:is_admin(User, Pass); +    nil ->          false      end, @@ -214,7 +237,7 @@ check_is_admin(Req) ->      false ->          case couch_server:has_admins() of          true -> -            throw(admin_auth_error); +            throw(admin_authorization_error);          false ->              % if no admins, then everyone is admin! Yay, admin party!              ok @@ -265,7 +288,6 @@ end_json_response(Resp) ->      send_chunk(Resp, []). -  send_error(Req, bad_request) ->      send_error(Req, 400, <<"bad_request">>, <<>>);  send_error(Req, {bad_request, Reason}) -> @@ -276,13 +298,18 @@ send_error(Req, {not_found, Reason}) ->      send_error(Req, 404, <<"not_found">>, Reason);  send_error(Req, conflict) ->      send_error(Req, 412, <<"conflict">>, <<"Document update conflict.">>); -send_error(Req, admin_auth_error) -> +send_error(Req, admin_authorization_error) ->      send_json(Req, 401, -        [{"WWW-Authenticate", "Basic realm=\"admin\""}], -        {[{<<"error">>,  <<"auth_error">>}, +        [{"WWW-Authenticate", "Basic realm=\"administrator\""}], +        {[{<<"error">>,  <<"authorization">>},           {<<"reason">>, <<"Admin user name and password required">>}]}); -send_error(Req, {doc_validation, Msg}) -> -    send_error(Req, 406, <<"doc_validation">>, Msg); +send_error(Req, {user_error, {Props}}) -> +    {Headers} = proplists:get_value(<<"headers">>, Props, {[]}), +    send_json(Req, +        proplists:get_value(<<"http_status">>, Props, 500), +        Headers, +        {[{<<"error">>, proplists:get_value(<<"error">>, Props)}, +            {<<"reason">>, proplists:get_value(<<"reason">>, Props)}]});  send_error(Req, file_exists) ->      send_error(Req, 409, <<"file_exists">>, <<"The database could not be "          "created, the file already exists.">>); diff --git a/src/couchdb/couch_httpd_db.erl b/src/couchdb/couch_httpd_db.erl index 8b8a7df0..bc8e2019 100644 --- a/src/couchdb/couch_httpd_db.erl +++ b/src/couchdb/couch_httpd_db.erl @@ -43,7 +43,7 @@ handle_request(#httpd{path_parts=[DbName|RestParts],method=Method,  create_db_req(Req, DbName) ->      ok = couch_httpd:check_is_admin(Req), -    case couch_server:create(DbName, []) of +    case couch_server:create(DbName, [{creds, couch_httpd:creds(Req)}]) of      {ok, Db} ->          couch_db:close(Db),          send_json(Req, 201, {[{ok, true}]}); @@ -53,7 +53,7 @@ create_db_req(Req, DbName) ->  delete_db_req(Req, DbName) ->      ok = couch_httpd:check_is_admin(Req), -    case couch_server:delete(DbName) of +    case couch_server:delete(DbName, [{creds, couch_httpd:creds(Req)}]) of      ok ->          send_json(Req, 200, {[{ok, true}]});      Error -> @@ -61,7 +61,7 @@ delete_db_req(Req, DbName) ->      end.  do_db_req(#httpd{path_parts=[DbName|_]}=Req, Fun) -> -    case couch_db:open(DbName, []) of +    case couch_db:open(DbName, [{creds, couch_httpd:creds(Req)}]) of      {ok, Db} ->          try              Fun(Req, Db) @@ -129,7 +129,7 @@ db_req(#httpd{method='POST',path_parts=[_,<<"_bulk_docs">>]}=Req, Db) ->              _ -> []          end,          Docs = [couch_doc:from_json_obj(JsonObj) || JsonObj <- DocsArray], -        ok = couch_db:save_docs(Db, Docs, Options), +        ok = couch_db:update_docs(Db, Docs, Options, false),          send_json(Req, 201, {[              {ok, true}          ]}) diff --git a/src/couchdb/couch_key_tree.erl b/src/couchdb/couch_key_tree.erl index 3a05fd4d..82c42265 100644 --- a/src/couchdb/couch_key_tree.erl +++ b/src/couchdb/couch_key_tree.erl @@ -13,7 +13,7 @@  -module(couch_key_tree).  -export([merge/2, find_missing/2, get_key_leafs/2, get_full_key_paths/2, get/2]). --export([map/2, get_all_leafs/1, get_leaf_keys/1, count_leafs/1, remove_leafs/2]). +-export([map/2, get_all_leafs/1, get_leaf_keys/1, count_leafs/1, remove_leafs/2,get_all_leafs_full/1]).  % a key tree looks like this:  % Tree -> [] or [{Key, Value, ChildTree} | SiblingTree] @@ -137,6 +137,16 @@ get_full_key_paths([{KeyId, Value, SubTree} | RestTree], KeysToGet, KeyPathAcc)      {KeysGotten2, KeysRemaining2} = get_full_key_paths(RestTree, KeysRemaining, KeyPathAcc),      {CurrentNodeResult ++ KeysGotten ++ KeysGotten2, KeysRemaining2}. +get_all_leafs_full(Tree) -> +    get_all_leafs_full(Tree, []). +     +get_all_leafs_full([], _KeyPathAcc) -> +    []; +get_all_leafs_full([{KeyId, Value, []} | RestTree], KeyPathAcc) -> +    [[{KeyId, Value} | KeyPathAcc] | get_all_leafs_full(RestTree, KeyPathAcc)]; +get_all_leafs_full([{KeyId, Value, SubTree} | RestTree], KeyPathAcc) -> +    get_all_leafs_full(SubTree, [{KeyId, Value} | KeyPathAcc]) ++ get_all_leafs_full(RestTree, KeyPathAcc). +  get_all_leafs(Tree) ->      get_all_leafs(Tree, []). diff --git a/src/couchdb/couch_query_servers.erl b/src/couchdb/couch_query_servers.erl index e9f309a2..141c9406 100644 --- a/src/couchdb/couch_query_servers.erl +++ b/src/couchdb/couch_query_servers.erl @@ -17,7 +17,7 @@  -export([init/1, terminate/2, handle_call/3, handle_cast/2, handle_info/2,code_change/3,stop/0]).  -export([start_doc_map/2, map_docs/2, stop_doc_map/1]). --export([reduce/3, rereduce/3]). +-export([reduce/3, rereduce/3,validate_doc_update/5]).  % -export([test/0]).  -include("couch_db.hrl"). @@ -46,10 +46,10 @@ readline(Port, Acc) ->          lists:reverse(Acc, Data);      {Port, Err} ->          catch port_close(Port), -        throw({map_process_error, Err}) +        throw({external_process_error, Err})      after Timeout ->          catch port_close(Port), -        throw({map_process_error, "map function timed out"}) +        throw({external_process_error, "External process timed out"})      end.  read_json(Port) -> @@ -174,6 +174,25 @@ reduce(Lang, RedSrcs, KVs) ->      return_linked_port(Lang, Port),      {ok, Results}. +validate_doc_update(Lang, FunSrc, EditDoc, DiskDoc, Ctx) -> +    Port = get_linked_port(Lang), +    JsonEditDoc = couch_doc:to_json_obj(EditDoc, [revs]), +    JsonDiskDoc = +    if DiskDoc == nil -> +        null; +    true ->  +        couch_doc:to_json_obj(EditDoc, [revs]) +    end, +    try prompt(Port,  +            [<<"validate">>, FunSrc, JsonEditDoc, JsonDiskDoc, Ctx]) of +    1 -> +        ok; +    {ErrorObject} -> +        {user_error, +           {ErrorObject}} +    after +        return_linked_port(Lang, Port) +    end.  init([]) -> diff --git a/src/couchdb/couch_rep.erl b/src/couchdb/couch_rep.erl index c91364c8..d0a12fc7 100644 --- a/src/couchdb/couch_rep.erl +++ b/src/couchdb/couch_rep.erl @@ -209,7 +209,7 @@ save_docs_loop(DbTarget, DocsWritten) ->      receive      {Src, docs, Docs} ->          Src ! got_it, -        ok = save_docs(DbTarget, Docs, []), +        ok = update_docs(DbTarget, Docs, [], false),          save_docs_loop(DbTarget, DocsWritten + length(Docs));      {Src, shutdown} ->          Src ! {done, self(), [{<<"docs_written">>, DocsWritten}]} @@ -313,16 +313,16 @@ update_doc(DbUrl, #doc{id=DocId}=Doc, _Options) when is_list(DbUrl) ->  update_doc(Db, Doc, Options) ->      couch_db:update_doc(Db, Doc, Options). -save_docs(_, [], _) -> +update_docs(_, [], _, _) ->      ok; -save_docs(DbUrl, Docs, []) when is_list(DbUrl) -> +update_docs(DbUrl, Docs, [], NewEdits) when is_list(DbUrl) ->      JsonDocs = [couch_doc:to_json_obj(Doc, [revs,attachments]) || Doc <- Docs],      {Returned} = -        do_http_request(DbUrl ++ "_bulk_docs", post, {[{new_edits, false}, {docs, JsonDocs}]}), +        do_http_request(DbUrl ++ "_bulk_docs", post, {[{new_edits, NewEdits}, {docs, JsonDocs}]}),      true = proplists:get_value(<<"ok">>, Returned),      ok; -save_docs(Db, Docs, Options) -> -    couch_db:save_docs(Db, Docs, Options). +update_docs(Db, Docs, Options, NewEdits) -> +    couch_db:update_docs(Db, Docs, Options, NewEdits).  open_doc(DbUrl, DocId, []) when is_list(DbUrl) -> diff --git a/src/couchdb/couch_server.erl b/src/couchdb/couch_server.erl index 02a2fbe4..b0d03a53 100644 --- a/src/couchdb/couch_server.erl +++ b/src/couchdb/couch_server.erl @@ -15,7 +15,7 @@  -behaviour(application).  -export([start/0,start/1,start/2,stop/0,stop/1,restart/0]). --export([open/2,create/2,delete/1,all_databases/0,get_version/0]). +-export([open/2,create/2,delete/2,all_databases/0,get_version/0]).  -export([init/1, handle_call/3,sup_start_link/0]).  -export([handle_cast/2,code_change/3,handle_info/2,terminate/2]).  -export([dev_start/0,remote_restart/0,is_admin/2,has_admins/0]). @@ -71,8 +71,8 @@ open(DbName, Options) ->  create(DbName, Options) ->      gen_server:call(couch_server, {create, DbName, Options}). -delete(DbName) -> -    gen_server:call(couch_server, {delete, DbName}). +delete(DbName, Options) -> +    gen_server:call(couch_server, {delete, DbName, Options}).  remote_restart() ->      gen_server:call(couch_server, remote_restart). @@ -202,6 +202,7 @@ handle_call(get_root, _From, #server{root_dir=Root}=Server) ->      {reply, {ok, Root}, Server};  handle_call({open, DbName, Options}, {FromPid,_}, Server) ->      DbNameList = binary_to_list(DbName), +    UserCreds = proplists:get_value(creds, Options, nil),      case check_dbname(Server, DbNameList) of      ok ->          Filepath = get_full_filename(Server, DbNameList), @@ -217,7 +218,7 @@ handle_call({open, DbName, Options}, {FromPid,_}, Server) ->                      true = ets:insert(couch_dbs_by_lru, {LruTime, DbName}),                      DbsOpen = Server2#server.current_dbs_open + 1,                      {reply, -                        couch_db:open_ref_counted(MainPid, FromPid), +                        couch_db:open_ref_counted(MainPid, FromPid, UserCreds),                          Server2#server{current_dbs_open=DbsOpen}};                  Error ->                      {reply, Error, Server2} @@ -229,13 +230,16 @@ handle_call({open, DbName, Options}, {FromPid,_}, Server) ->              true = ets:insert(couch_dbs_by_name, {DbName, {MainPid, LruTime}}),              true = ets:delete(couch_dbs_by_lru, PrevLruTime),              true = ets:insert(couch_dbs_by_lru, {LruTime, DbName}), -            {reply, couch_db:open_ref_counted(MainPid, FromPid), Server} +            {reply, +                couch_db:open_ref_counted(MainPid, FromPid, UserCreds), +                Server}          end;      Error ->          {reply, Error, Server}      end;  handle_call({create, DbName, Options}, {FromPid,_}, Server) ->      DbNameList = binary_to_list(DbName), +    UserCreds = proplists:get_value(creds, Options, nil),      case check_dbname(Server, DbNameList) of      ok ->          Filepath = get_full_filename(Server, DbNameList), @@ -251,7 +255,7 @@ handle_call({create, DbName, Options}, {FromPid,_}, Server) ->                  DbsOpen = Server#server.current_dbs_open + 1,                  couch_db_update_notifier:notify({created, DbName}),                  {reply, -                    couch_db:open_ref_counted(MainPid, FromPid),  +                    couch_db:open_ref_counted(MainPid, FromPid, UserCreds),                       Server#server{current_dbs_open=DbsOpen}};              Error ->                  {reply, Error, Server} @@ -262,8 +266,9 @@ handle_call({create, DbName, Options}, {FromPid,_}, Server) ->      Error ->          {reply, Error, Server}      end; -handle_call({delete, DbName}, _From, Server) -> +handle_call({delete, DbName, Options}, _From, Server) ->      DbNameList = binary_to_list(DbName), +    _UserCreds = proplists:get_value(creds, Options, nil),      case check_dbname(Server, DbNameList) of      ok ->          FullFilepath = get_full_filename(Server, DbNameList), @@ -306,10 +311,10 @@ handle_cast(Msg, _Server) ->  code_change(_OldVsn, State, _Extra) ->      {ok, State}. -handle_info({'EXIT', Pid, _Reason}, Server) -> +handle_info({'EXIT', Pid, _Reason}, #server{current_dbs_open=DbsOpen}=Server) ->      [{Pid, DbName}] = ets:lookup(couch_dbs_by_pid, Pid),      true = ets:delete(couch_dbs_by_pid, Pid),      true = ets:delete(couch_dbs_by_name, DbName), -    {noreply, Server}; +    {noreply, Server#server{current_dbs_open=DbsOpen-1}};  handle_info(Info, _Server) ->      exit({unknown_message, Info}). | 
