diff options
Diffstat (limited to 'src/couchdb/couch_db.erl')
-rw-r--r-- | src/couchdb/couch_db.erl | 395 |
1 files changed, 277 insertions, 118 deletions
diff --git a/src/couchdb/couch_db.erl b/src/couchdb/couch_db.erl index 92026ff1..46de8745 100644 --- a/src/couchdb/couch_db.erl +++ b/src/couchdb/couch_db.erl @@ -17,6 +17,7 @@ -export([open_ref_counted/2,is_idle/1,monitor/1,count_changes_since/2]). -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([set_revs_limit/2,get_revs_limit/1]). -export([get_missing_revs/2,name/1,doc_to_tree/1,get_update_seq/1,get_committed_update_seq/1]). -export([enum_docs/4,enum_docs/5,enum_docs_since/4,enum_docs_since/5]). -export([enum_docs_since_reduce_to_count/1,enum_docs_reduce_to_count/1]). @@ -184,22 +185,42 @@ get_db_info(Db) -> ], {ok, InfoList}. +check_is_admin(#db{admins=Admins, user_ctx=#user_ctx{name=Name,roles=Roles}}) -> + DbAdmins = [<<"_admin">> | Admins], + case DbAdmins -- [Name | Roles] of + DbAdmins -> % same list, not an admin + throw({unauthorized, <<"You are not a db or server admin.">>}); + _ -> + ok + end. + get_admins(#db{admins=Admins}) -> Admins. -set_admins(#db{update_pid=UpdatePid,user_ctx=Ctx}, - Admins) when is_list(Admins) -> - case gen_server:call(UpdatePid, {set_admins, Admins, Ctx}, infinity) of - ok -> ok; - Error -> throw(Error) - end. +set_admins(#db{update_pid=Pid}=Db, Admins) when is_list(Admins) -> + check_is_admin(Db), + gen_server:call(Pid, {set_admins, Admins}, infinity). + + +get_revs_limit(#db{revs_limit=Limit}) -> + Limit. + +set_revs_limit(#db{update_pid=Pid}=Db, Limit) when Limit > 0 -> + check_is_admin(Db), + gen_server:call(Pid, {set_revs_limit, Limit}, infinity); +set_revs_limit(_Db, _Limit) -> + throw(invalid_revs_limit). name(#db{name=Name}) -> Name. update_doc(Db, Doc, Options) -> - {ok, [NewRev]} = update_docs(Db, [Doc], Options), - {ok, NewRev}. + case update_docs(Db, [Doc], Options) of + {ok, [{ok, NewRev}]} -> + {ok, NewRev}; + {ok, [Error]} -> + throw(Error) + end. update_docs(Db, Docs) -> update_docs(Db, Docs, []). @@ -227,146 +248,283 @@ group_alike_docs([Doc|Rest], [Bucket|RestBuckets]) -> validate_doc_update(#db{user_ctx=UserCtx, admins=Admins}, - #doc{id= <<"_design/",_/binary>>}=Doc, _GetDiskDocFun) -> + #doc{id= <<"_design/",_/binary>>}, _GetDiskDocFun) -> UserNames = [UserCtx#user_ctx.name | UserCtx#user_ctx.roles], % if the user is a server admin or db admin, allow the save case length(UserNames -- [<<"_admin">> | Admins]) == length(UserNames) of true -> % not an admin - throw({unauthorized, <<"You are not a server or database admin.">>}); + {unauthorized, <<"You are not a server or database admin.">>}; false -> - Doc + ok end; -validate_doc_update(#db{validate_doc_funs=[]}, Doc, _GetDiskDocFun) -> - Doc; -validate_doc_update(_Db, #doc{id= <<"_local/",_/binary>>}=Doc, _GetDiskDocFun) -> - Doc; +validate_doc_update(#db{validate_doc_funs=[]}, _Doc, _GetDiskDocFun) -> + ok; +validate_doc_update(_Db, #doc{id= <<"_local/",_/binary>>}, _GetDiskDocFun) -> + ok; validate_doc_update(#db{name=DbName,user_ctx=Ctx}=Db, Doc, GetDiskDocFun) -> DiskDoc = GetDiskDocFun(), JsonCtx = {[{<<"db">>, DbName}, {<<"name">>,Ctx#user_ctx.name}, {<<"roles">>,Ctx#user_ctx.roles}]}, - [case Fun(Doc, DiskDoc, JsonCtx) of - ok -> ok; - Error -> throw(Error) - end || Fun <- Db#db.validate_doc_funs], - Doc. + try [case Fun(Doc, DiskDoc, JsonCtx) of + ok -> ok; + Error -> throw(Error) + end || Fun <- Db#db.validate_doc_funs], + ok + catch + throw:Error -> + Error + end. -prep_and_validate_new_edit(Db, #doc{id=Id,revs=[NewRev|PrevRevs]}=Doc, +prep_and_validate_update(Db, #doc{id=Id,revs={RevStart, [_NewRev|PrevRevs]}}=Doc, OldFullDocInfo, LeafRevsDict) -> case PrevRevs of [PrevRev|_] -> - case dict:find(PrevRev, LeafRevsDict) of - {ok, {Deleted, Sp, DiskRevs}} -> - Doc2 = Doc#doc{revs=[NewRev|DiskRevs]}, - case couch_doc:has_stubs(Doc2) of + case dict:find({RevStart-1, PrevRev}, LeafRevsDict) of + {ok, {Deleted, DiskSp, DiskRevs}} -> + case couch_doc:has_stubs(Doc) of true -> - DiskDoc = make_doc(Db, Id, Deleted, Sp, DiskRevs), - Doc3 = couch_doc:merge_stubs(Doc2, DiskDoc), - validate_doc_update(Db, Doc3, fun() -> DiskDoc end); + DiskDoc = make_doc(Db, Id, Deleted, DiskSp, DiskRevs), + Doc2 = couch_doc:merge_stubs(Doc, DiskDoc), + {validate_doc_update(Db, Doc2, fun() -> DiskDoc end), Doc2}; false -> - LoadDiskDoc = fun() -> make_doc(Db,Id,Deleted,Sp,DiskRevs) end, - validate_doc_update(Db, Doc2, LoadDiskDoc) + LoadDiskDoc = fun() -> make_doc(Db,Id,Deleted,DiskSp,DiskRevs) end, + {validate_doc_update(Db, Doc, LoadDiskDoc), Doc} end; error -> - throw(conflict) + {conflict, Doc} end; [] -> % new doc, and we have existing revs. if OldFullDocInfo#full_doc_info.deleted -> % existing docs are deletions - validate_doc_update(Db, Doc, nil); + {validate_doc_update(Db, Doc, fun() -> nil end), Doc}; true -> - throw(conflict) + {conflict, Doc} end end. + + +prep_and_validate_updates(_Db, [], [], AccPrepped, AccFatalErrors) -> + {AccPrepped, AccFatalErrors}; +prep_and_validate_updates(Db, [DocBucket|RestBuckets], [not_found|RestLookups], AccPrepped, AccErrors) -> + [#doc{id=Id}|_]=DocBucket, + % no existing revs are known, + {PreppedBucket, AccErrors3} = lists:foldl( + fun(#doc{revs=Revs}=Doc, {AccBucket, AccErrors2}) -> + case Revs of + {Pos, [NewRev,_OldRev|_]} -> + % old revs specified but none exist, a conflict + {AccBucket, [{{Id, {Pos, NewRev}}, conflict} | AccErrors2]}; + {Pos, [NewRev]} -> + case validate_doc_update(Db, Doc, fun() -> nil end) of + ok -> + {[Doc | AccBucket], AccErrors2}; + Error -> + {AccBucket, [{{Id, {Pos, NewRev}}, Error} | AccErrors2]} + end + end + end, + {[], AccErrors}, DocBucket), + + prep_and_validate_updates(Db, RestBuckets, RestLookups, + [PreppedBucket | AccPrepped], AccErrors3); +prep_and_validate_updates(Db, [DocBucket|RestBuckets], + [{ok, #full_doc_info{rev_tree=OldRevTree}=OldFullDocInfo}|RestLookups], + AccPrepped, AccErrors) -> + Leafs = couch_key_tree:get_all_leafs(OldRevTree), + LeafRevsDict = dict:from_list([{{Start, RevId}, {Deleted, Sp, Revs}} || + {{Deleted, Sp}, {Start, [RevId|_]}=Revs} <- Leafs]), + {PreppedBucket, AccErrors3} = lists:foldl( + fun(Doc, {Docs2Acc, AccErrors2}) -> + case prep_and_validate_update(Db, Doc, OldFullDocInfo, + LeafRevsDict) of + {ok, Doc} -> + {[Doc | Docs2Acc], AccErrors2}; + {Error, #doc{id=Id,revs={Pos, [NewRev|_]}}} -> + % Record the error + {Docs2Acc, [{{Id, {Pos, NewRev}}, Error} |AccErrors2]} + end + end, + {[], AccErrors}, DocBucket), + prep_and_validate_updates(Db, RestBuckets, RestLookups, [PreppedBucket | AccPrepped], AccErrors3). + + update_docs(#db{update_pid=UpdatePid}=Db, Docs, Options) -> - update_docs(#db{update_pid=UpdatePid}=Db, Docs, Options, true). + update_docs(#db{update_pid=UpdatePid}=Db, Docs, Options, interactive_edit). -update_docs(Db, Docs, Options, false) -> + +prep_and_validate_replicated_updates(_Db, [], [], AccPrepped, AccErrors) -> + Errors2 = [{{Id, {Pos, Rev}}, Error} || + {#doc{id=Id,revs={Pos,[Rev|_]}}, Error} <- AccErrors], + {lists:reverse(AccPrepped), lists:reverse(Errors2)}; +prep_and_validate_replicated_updates(Db, [Bucket|RestBuckets], [OldInfo|RestOldInfo], AccPrepped, AccErrors) -> + case OldInfo of + not_found -> + {ValidatedBucket, AccErrors3} = lists:foldl( + fun(Doc, {AccPrepped2, AccErrors2}) -> + case validate_doc_update(Db, Doc, fun() -> nil end) of + ok -> + {[Doc | AccPrepped2], AccErrors2}; + Error -> + {AccPrepped2, [{Doc, Error} | AccErrors2]} + end + end, + {[], AccErrors}, Bucket), + prep_and_validate_replicated_updates(Db, RestBuckets, RestOldInfo, [ValidatedBucket | AccPrepped], AccErrors3); + {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 + end, + OldTree, Bucket), + Leafs = couch_key_tree:get_all_leafs_full(NewRevTree), + LeafRevsFullDict = dict:from_list( [{{Start, RevId}, FullPath} || {Start, [{RevId, _}|_]}=FullPath <- Leafs]), + {ValidatedBucket, AccErrors3} = + lists:foldl( + fun(#doc{id=Id,revs={Pos, [RevId|_]}}=Doc, {AccValidated, AccErrors2}) -> + case dict:find({Pos, RevId}, LeafRevsFullDict) of + {ok, {Start, Path}} -> + % our unflushed doc is a leaf node. Go back on the path + % to find the previous rev that's on disk. + PrevRevResult = + case couch_doc:has_stubs(Doc) of + true -> + [_PrevRevFull | [PrevRevFull | _]=PrevPath] = Path, + case PrevRevFull of + {_RevId, ?REV_MISSING} -> + conflict; + {RevId, {IsDel, DiskSp}} -> + DiskDoc = make_doc(Db, Id, IsDel, DiskSp, PrevPath), + Doc2 = couch_doc:merge_stubs(Doc, DiskDoc), + {ok, Doc2, fun() -> DiskDoc end} + end; + false -> + {ok, Doc, + fun() -> + make_first_doc_on_disk(Db,Id,Start-1, tl(Path)) + end} + end, + case PrevRevResult of + {ok, NewDoc, LoadPrevRevFun} -> + case validate_doc_update(Db, NewDoc, LoadPrevRevFun) of + ok -> + {[NewDoc | AccValidated], AccErrors2}; + Error -> + {AccValidated, [{NewDoc, Error} | AccErrors2]} + end; + Error -> + {AccValidated, [{Doc, Error} | AccErrors2]} + end; + _ -> + % this doc isn't a leaf or already exists in the tree. + % ignore but consider it a success. + {AccValidated, AccErrors2} + end + end, + {[], AccErrors}, Bucket), + prep_and_validate_replicated_updates(Db, RestBuckets, RestOldInfo, [ValidatedBucket | AccPrepped], AccErrors3) + end. + +update_docs(Db, Docs, Options, replicated_changes) -> couch_stats_collector:increment({couchdb, database_writes}), DocBuckets = group_alike_docs(Docs), - Ids = [Id || [#doc{id=Id}|_] <- DocBuckets], - ExistingDocs = get_full_doc_infos(Db, Ids), + case (Db#db.validate_doc_funs /= []) orelse + lists:any( + fun(#doc{id= <<?DESIGN_DOC_PREFIX, _/binary>>}) -> true; + (_) -> false + end, Docs) of + true -> + 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); + {DocBuckets2, DocErrors} = + prep_and_validate_replicated_updates(Db, DocBuckets, ExistingDocs, [], []), + DocBuckets3 = [Bucket || [_|_]=Bucket <- DocBuckets2]; % remove empty buckets + false -> + DocErrors = [], + DocBuckets3 = DocBuckets + end, + {ok, []} = write_and_commit(Db, DocBuckets3, [merge_conflicts | Options]), + {ok, DocErrors}; -update_docs(Db, Docs, Options, true) -> +update_docs(Db, Docs, Options, interactive_edit) -> couch_stats_collector:increment({couchdb, database_writes}), - + AllOrNothing = lists:member(all_or_nothing, Options), % go ahead and generate the new revision ids for the documents. Docs2 = lists:map( - fun(#doc{id=Id,revs=Revs}=Doc) -> + fun(#doc{id=Id,revs={Start, RevIds}}=Doc) -> case Id of <<?LOCAL_DOC_PREFIX, _/binary>> -> - Rev = case Revs of [] -> 0; [Rev0|_] -> list_to_integer(binary_to_list(Rev0)) end, - Doc#doc{revs=[list_to_binary(integer_to_list(Rev + 1))]}; + Rev = case RevIds of [] -> 0; [Rev0|_] -> list_to_integer(?b2l(Rev0)) end, + Doc#doc{revs={Start, [?l2b(integer_to_list(Rev + 1))]}}; _ -> - Doc#doc{revs=[list_to_binary(integer_to_list(couch_util:rand32())) | Revs]} + Doc#doc{revs={Start+1, [?l2b(integer_to_list(couch_util:rand32())) | RevIds]}} end end, Docs), DocBuckets = group_alike_docs(Docs2), - Ids = [Id || [#doc{id=Id}|_] <- DocBuckets], - - % lookup the doc by id and get the most recent - - ExistingDocs = get_full_doc_infos(Db, Ids), - DocBuckets2 = lists:zipwith( - fun(Bucket, not_found) -> - % no existing revs on disk, make sure no old revs specified. - [throw(conflict) || #doc{revs=[_NewRev, _OldRev | _]} <- 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]), - [prep_and_validate_new_edit(Db, Doc, OldFullDocInfo, LeafRevsDict) || Doc <- Bucket] + case (Db#db.validate_doc_funs /= []) orelse + lists:any( + fun(#doc{id= <<?DESIGN_DOC_PREFIX, _/binary>>}) -> + true; + (#doc{attachments=Atts}) -> + Atts /= [] + end, Docs) of + true -> + % lookup the doc by id and get the most recent + Ids = [Id || [#doc{id=Id}|_] <- DocBuckets], + ExistingDocInfos = get_full_doc_infos(Db, Ids), + + {DocBucketsPrepped, Failures} = + case AllOrNothing of + true -> + prep_and_validate_replicated_updates(Db, DocBuckets, + ExistingDocInfos, [], []); + false -> + prep_and_validate_updates(Db, DocBuckets, ExistingDocInfos, [], []) end, - DocBuckets, ExistingDocs), - ok = write_and_commit(Db, DocBuckets2, [new_edits | Options]), - {ok, [NewRev ||#doc{revs=[NewRev|_]} <- Docs2]}. + + % strip out any empty buckets + DocBuckets2 = [Bucket || [_|_] = Bucket <- DocBucketsPrepped]; + false -> + Failures = [], + DocBuckets2 = DocBuckets + end, + if (AllOrNothing) and (Failures /= []) -> + {aborted, Failures}; + true -> + Options2 = if AllOrNothing -> [merge_conflicts]; + true -> [] end ++ Options, + {ok, CommitFailures} = write_and_commit(Db, DocBuckets2, Options2), + FailDict = dict:from_list(CommitFailures ++ Failures), + % the output for each is either {ok, NewRev} or Error + {ok, lists:map( + fun(#doc{id=Id,revs={Pos, [NewRevId|_]}}) -> + case dict:find({Id, {Pos, NewRevId}}, FailDict) of + {ok, Error} -> + Error; + error -> + {ok, {Pos, NewRevId}} + end + end, Docs2)} + end. % Returns the first available document on disk. Input list is a full rev path % for the doc. -make_first_doc_on_disk(_Db, _Id, []) -> +make_first_doc_on_disk(_Db, _Id, _Pos, []) -> 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) -> +make_first_doc_on_disk(Db, Id, Pos, [{_Rev, ?REV_MISSING}|RestPath]) -> + make_first_doc_on_disk(Db, Id, Pos - 1, RestPath); +make_first_doc_on_disk(Db, Id, Pos, [{_Rev, {IsDel, Sp}} |_]=DocPath) -> Revs = [Rev || {Rev, _} <- DocPath], - make_doc(Db, Id, IsDel, Sp, Revs). + make_doc(Db, Id, IsDel, Sp, {Pos, Revs}). write_and_commit(#db{update_pid=UpdatePid, user_ctx=Ctx}=Db, DocBuckets, @@ -374,20 +532,18 @@ write_and_commit(#db{update_pid=UpdatePid, user_ctx=Ctx}=Db, DocBuckets, % 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; + {ok, Conflicts} -> {ok, Conflicts}; retry -> % This can happen if the db file we wrote to was swapped out by - % compaction. Retry writing to the current file + % compaction. Retry by reopening the db and writing to the current file {ok, Db2} = open_ref_counted(Db#db.main_pid, Ctx), 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, DocBuckets3, Options}, infinity) of - ok -> ok; - Else -> throw(Else) - end; - Else-> - throw(Else) + {ok, Conflicts} -> {ok, Conflicts}; + retry -> throw({update_error, compaction_retry}) + end end. @@ -506,7 +662,7 @@ enum_docs_reduce_to_count(Reds) -> {Count, _DelCount} = couch_btree:final_reduce( fun couch_db_updater:btree_by_id_reduce/2, Reds), Count. - + count_changes_since(Db, SinceSeq) -> {ok, Changes} = couch_btree:fold_reduce(Db#db.docinfo_by_seq_btree, @@ -518,7 +674,7 @@ count_changes_since(Db, SinceSeq) -> end, ok), Changes. - + enum_docs_since(Db, SinceSeq, Direction, InFun, Ctx) -> couch_btree:fold(Db#db.docinfo_by_seq_btree, SinceSeq + 1, Direction, InFun, Ctx). @@ -594,11 +750,11 @@ open_doc_revs_int(Db, IdRevs, Options) -> end end, FoundResults = - lists:map(fun({Rev, Value, FoundRevPath}) -> + lists:map(fun({Value, {Pos, [Rev|_]}=FoundRevPath}) -> case Value of ?REV_MISSING -> % we have the rev in our list but know nothing about it - {{not_found, missing}, Rev}; + {{not_found, missing}, {Pos, Rev}}; {IsDeleted, SummaryPtr} -> {ok, make_doc(Db, Id, IsDeleted, SummaryPtr, FoundRevPath)} end @@ -616,18 +772,18 @@ open_doc_revs_int(Db, IdRevs, Options) -> open_doc_int(Db, <<?LOCAL_DOC_PREFIX, _/binary>> = Id, _Options) -> case couch_btree:lookup(Db#db.local_docs_btree, [Id]) of [{ok, {_, {Rev, BodyData}}}] -> - {ok, #doc{id=Id, revs=[list_to_binary(integer_to_list(Rev))], body=BodyData}}; + {ok, #doc{id=Id, revs={0, [list_to_binary(integer_to_list(Rev))]}, body=BodyData}}; [not_found] -> {not_found, missing} end; -open_doc_int(Db, #doc_info{id=Id,rev=Rev,deleted=IsDeleted,summary_pointer=Sp}=DocInfo, Options) -> - Doc = make_doc(Db, Id, IsDeleted, Sp, [Rev]), +open_doc_int(Db, #doc_info{id=Id,rev={Pos,RevId},deleted=IsDeleted,summary_pointer=Sp}=DocInfo, Options) -> + Doc = make_doc(Db, Id, IsDeleted, Sp, {Pos,[RevId]}), {ok, Doc#doc{meta=doc_meta_info(DocInfo, [], Options)}}; open_doc_int(Db, #full_doc_info{id=Id,rev_tree=RevTree}=FullDocInfo, Options) -> #doc_info{deleted=IsDeleted,rev=Rev,summary_pointer=Sp} = DocInfo = couch_doc:to_doc_info(FullDocInfo), - {[{_Rev,_Value, Revs}], []} = couch_key_tree:get(RevTree, [Rev]), - Doc = make_doc(Db, Id, IsDeleted, Sp, Revs), + {[{_, RevPath}], []} = couch_key_tree:get(RevTree, [Rev]), + Doc = make_doc(Db, Id, IsDeleted, Sp, RevPath), {ok, Doc#doc{meta=doc_meta_info(DocInfo, RevTree, Options)}}; open_doc_int(Db, Id, Options) -> case get_full_doc_info(Db, Id) of @@ -641,9 +797,10 @@ doc_meta_info(DocInfo, RevTree, Options) -> case lists:member(revs_info, Options) of false -> []; true -> - {[RevPath],[]} = + {[{Pos, RevPath}],[]} = couch_key_tree:get_full_key_paths(RevTree, [DocInfo#doc_info.rev]), - [{revs_info, lists:map( + + [{revs_info, Pos, lists:map( fun({Rev, {true, _Sp}}) -> {Rev, deleted}; ({Rev, {false, _Sp}}) -> @@ -670,13 +827,15 @@ doc_meta_info(DocInfo, RevTree, Options) -> end. -doc_to_tree(Doc) -> - doc_to_tree(Doc, lists:reverse(Doc#doc.revs)). +doc_to_tree(#doc{revs={Start, RevIds}}=Doc) -> + [Tree] = doc_to_tree_simple(Doc, lists:reverse(RevIds)), + {Start - length(RevIds) + 1, Tree}. + -doc_to_tree(Doc, [RevId]) -> +doc_to_tree_simple(Doc, [RevId]) -> [{RevId, Doc, []}]; -doc_to_tree(Doc, [RevId | Rest]) -> - [{RevId, ?REV_MISSING, doc_to_tree(Doc, Rest)}]. +doc_to_tree_simple(Doc, [RevId | Rest]) -> + [{RevId, ?REV_MISSING, doc_to_tree_simple(Doc, Rest)}]. make_doc(Db, FullDocInfo) -> {#doc_info{id=Id,deleted=Deleted,summary_pointer=Sp}, RevPath} @@ -703,4 +862,4 @@ make_doc(#db{fd=Fd}=Db, Id, Deleted, BodySp, RevisionPath) -> }. -
\ No newline at end of file + |